From 0816dedaea76fa9bb6e22a27f9c001ef0c4e2962 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 22 Oct 2021 15:56:09 +0300 Subject: [PATCH 001/164] Introduce memory overcommit --- programs/server/Server.cpp | 2 + src/Common/MemoryTracker.cpp | 61 +++++++++++--- src/Common/MemoryTracker.h | 14 ++++ src/Common/OvercommitTracker.cpp | 60 ++++++++++++++ src/Common/OvercommitTracker.h | 131 +++++++++++++++++++++++++++++++ src/Core/Settings.h | 2 + src/Interpreters/Context.cpp | 6 +- src/Interpreters/Context.h | 3 + src/Interpreters/ProcessList.cpp | 14 +++- src/Interpreters/ProcessList.h | 16 ++++ 10 files changed, 294 insertions(+), 15 deletions(-) create mode 100644 src/Common/OvercommitTracker.cpp create mode 100644 src/Common/OvercommitTracker.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 29845f23d92..7be5aaea429 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -22,6 +22,7 @@ #include #include #include +#include "Common/MemoryTracker.h" #include #include #include @@ -840,6 +841,7 @@ if (ThreadFuzzer::instance().isEffective()) total_memory_tracker.setHardLimit(max_server_memory_usage); total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + total_memory_tracker.setOvercommitTracker(global_context->getGlobalOvercommitTracker()); // FIXME logging-related things need synchronization -- see the 'Logger * log' saved // in a lot of places. For now, disable updating log configuration without server restart. diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 013005442be..61aa470e2d4 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -2,11 +2,13 @@ #include #include "Common/TraceCollector.h" +#include "Common/VariableContext.h" #include #include #include #include #include +#include #include #include @@ -99,6 +101,17 @@ MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : MemoryTracker::~MemoryTracker() { + if (level == VariableContext::Process) + { + auto * loaded_next = getParent(); + while (loaded_next != nullptr) + { + if (auto * next_overcommit_tracker = loaded_next->overcommit_tracker) + next_overcommit_tracker->unsubscribe(this); + loaded_next = loaded_next->getParent(); + } + } + if ((level == VariableContext::Process || level == VariableContext::User) && peak) { try @@ -219,18 +232,29 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { - /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc - BlockerInThread untrack_lock(VariableContext::Global); - ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); - const auto * description = description_ptr.load(std::memory_order_relaxed); - throw DB::Exception( - DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}", - description ? " " : "", - description ? description : "", - formatReadableSizeWithBinarySuffix(will_be), - size, - formatReadableSizeWithBinarySuffix(current_hard_limit)); + bool need_to_throw = true; + if (overcommit_tracker) + need_to_throw = overcommit_tracker->needToStopQuery(this); + + if (need_to_throw) + { + /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc + BlockerInThread untrack_lock(VariableContext::Global); + ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); + const auto * description = description_ptr.load(std::memory_order_relaxed); + throw DB::Exception( + DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, + "Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}", + description ? " " : "", + description ? description : "", + formatReadableSizeWithBinarySuffix(will_be), + size, + formatReadableSizeWithBinarySuffix(current_hard_limit)); + } + else + { + will_be = amount.load(std::memory_order_relaxed); + } } bool peak_updated; @@ -335,10 +359,17 @@ void MemoryTracker::free(Int64 size) } +OvercommitRatio MemoryTracker::getOvercommitRatio() +{ + return { amount.load(std::memory_order_relaxed), soft_limit.load(std::memory_order_relaxed) }; +} + + void MemoryTracker::resetCounters() { amount.store(0, std::memory_order_relaxed); peak.store(0, std::memory_order_relaxed); + soft_limit.store(0, std::memory_order_relaxed); hard_limit.store(0, std::memory_order_relaxed); profiler_limit.store(0, std::memory_order_relaxed); } @@ -363,6 +394,12 @@ void MemoryTracker::set(Int64 to) } +void MemoryTracker::setSoftLimit(Int64 value) +{ + soft_limit.store(value, std::memory_order_relaxed); +} + + void MemoryTracker::setHardLimit(Int64 value) { hard_limit.store(value, std::memory_order_relaxed); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index ce0eef52e17..e055a347617 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -28,6 +28,9 @@ extern thread_local bool memory_tracker_always_throw_logical_error_on_allocation #define ALLOW_ALLOCATIONS_IN_SCOPE static_assert(true) #endif +struct OvercommitRatio; +struct OvercommitTracker; + /** Tracks memory consumption. * It throws an exception if amount of consumed memory become greater than certain limit. * The same memory tracker could be simultaneously used in different threads. @@ -37,6 +40,7 @@ class MemoryTracker private: std::atomic amount {0}; std::atomic peak {0}; + std::atomic soft_limit {0}; std::atomic hard_limit {0}; std::atomic profiler_limit {0}; @@ -58,6 +62,8 @@ private: /// This description will be used as prefix into log messages (if isn't nullptr) std::atomic description_ptr = nullptr; + OvercommitTracker * overcommit_tracker = nullptr; + bool updatePeak(Int64 will_be, bool log_memory_usage); void logMemoryUsage(Int64 current) const; @@ -105,6 +111,7 @@ public: return peak.load(std::memory_order_relaxed); } + void setSoftLimit(Int64 value); void setHardLimit(Int64 value); /** Set limit if it was not set. @@ -156,6 +163,13 @@ public: description_ptr.store(description, std::memory_order_relaxed); } + OvercommitRatio getOvercommitRatio(); + + void setOvercommitTracker(OvercommitTracker * tracker) noexcept + { + overcommit_tracker = tracker; + } + /// Reset the accumulated data void resetCounters(); diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp new file mode 100644 index 00000000000..a2c5ae0ba4b --- /dev/null +++ b/src/Common/OvercommitTracker.cpp @@ -0,0 +1,60 @@ +#include "OvercommitTracker.h" + +#include + +bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker) +{ + std::unique_lock lk(overcommit_m); + + pickQueryToExclude(); + assert(cancelation_state == QueryCancelationState::RUNNING); + if (tracker == picked_tracker) + return true; + + auto now = std::chrono::system_clock::now(); + return cv.wait_until(lk, now, [this]() + { + return cancelation_state == QueryCancelationState::NONE; + }); +} + +UserOvercommitTracker::UserOvercommitTracker(DB::ProcessListForUser * user_process_list_) + : user_process_list(user_process_list_) +{} + +void UserOvercommitTracker::pickQueryToExcludeImpl() +{ + MemoryTracker * current_tracker = nullptr; + OvercommitRatio current_ratio{0, 0}; + //TODO: ensure this container is not being modified + for (auto const & query : user_process_list->queries) + { + auto * memory_tracker = query.second->getMemoryTracker(); + auto ratio = memory_tracker->getOvercommitRatio(); + if (current_ratio < ratio) + { + current_tracker = memory_tracker; + current_ratio = ratio; + } + } + assert(current_tracker != nullptr); + picked_tracker = current_tracker; +} + +void GlobalOvercommitTracker::pickQueryToExcludeImpl() +{ + MemoryTracker * current_tracker = nullptr; + OvercommitRatio current_ratio{0, 0}; + process_list->processEachQueryStatus([&](DB::QueryStatus const & query) + { + auto * memory_tracker = query.getMemoryTracker(); + auto ratio = memory_tracker->getOvercommitRatio(); + if (current_ratio < ratio) + { + current_tracker = memory_tracker; + current_ratio = ratio; + } + }); + assert(current_tracker != nullptr); + picked_tracker = current_tracker; +} diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h new file mode 100644 index 00000000000..dcf6a9262ab --- /dev/null +++ b/src/Common/OvercommitTracker.h @@ -0,0 +1,131 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +struct OvercommitRatio +{ + OvercommitRatio(Int64 commited_, Int64 soft_limit_) + : commited(commited_) + , soft_limit(soft_limit_) + {} + + friend bool operator<(OvercommitRatio const& lhs, OvercommitRatio const& rhs) noexcept + { + return (lhs.commited / lhs.soft_limit) < (rhs.commited / rhs.soft_limit); + } + + Int64 commited; + Int64 soft_limit; +}; + +class MemoryTracker; + +struct OvercommitTracker : boost::noncopyable +{ + OvercommitTracker() = default; + + bool needToStopQuery(MemoryTracker * tracker); + + void unsubscribe(MemoryTracker * tracker) + { + std::unique_lock lk(overcommit_m); + if (tracker == picked_tracker) + { + picked_tracker = nullptr; + cancelation_state = QueryCancelationState::NONE; + cv.notify_all(); + } + } + + virtual ~OvercommitTracker() = default; + +protected: + virtual void pickQueryToExcludeImpl() = 0; + + mutable std::mutex overcommit_m; + mutable std::condition_variable cv; + + enum class QueryCancelationState + { + NONE, + RUNNING, + }; + + MemoryTracker * picked_tracker; + QueryCancelationState cancelation_state = QueryCancelationState::NONE; + +private: + + void pickQueryToExclude() + { + if (cancelation_state != QueryCancelationState::RUNNING) + { + pickQueryToExcludeImpl(); + cancelation_state = QueryCancelationState::RUNNING; + } + } + + friend struct BlockQueryIfMemoryLimit; +}; + +namespace DB +{ + class ProcessList; + struct ProcessListForUser; +} + +struct UserOvercommitTracker : OvercommitTracker +{ + explicit UserOvercommitTracker(DB::ProcessListForUser * user_process_list_); + + ~UserOvercommitTracker() override = default; + +protected: + void pickQueryToExcludeImpl() override final; + +private: + DB::ProcessListForUser * user_process_list; +}; + +struct GlobalOvercommitTracker : OvercommitTracker +{ + explicit GlobalOvercommitTracker(DB::ProcessList * process_list_) + : process_list(process_list_) + {} + + ~GlobalOvercommitTracker() override = default; + +protected: + void pickQueryToExcludeImpl() override final; + +private: + DB::ProcessList * process_list; +}; + +struct BlockQueryIfMemoryLimit +{ + BlockQueryIfMemoryLimit(OvercommitTracker const & overcommit_tracker) + : mutex(overcommit_tracker.overcommit_m) + , lk(mutex) + { + if (overcommit_tracker.cancelation_state == OvercommitTracker::QueryCancelationState::RUNNING) + { + //TODO: Add timeout + overcommit_tracker.cv.wait(lk, [&overcommit_tracker]() + { + return overcommit_tracker.cancelation_state == OvercommitTracker::QueryCancelationState::NONE; + }); + } + } + + ~BlockQueryIfMemoryLimit() = default; + +private: + std::mutex & mutex; + std::unique_lock lk; +}; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3863046b511..906f10ae653 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -349,7 +349,9 @@ class IColumn; M(OverflowMode, distinct_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ M(UInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \ + M(UInt64, max_guaranteed_memory_usage, 0, "Maximum guaranteed memory usage for processing of single query. It represents soft limit. Zero means unlimited.", 0) \ M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ + M(UInt64, max_guaranteed_memory_usage_for_user, 0, "Maximum guaranteed memory usage for processing all concurrently running queries for the user. It represents soft limit. Zero means unlimited.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f2d449c04d7..e6c5cfebe49 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -213,6 +213,7 @@ struct ContextSharedPart mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. ProcessList process_list; /// Executing queries at the moment. + GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. @@ -279,7 +280,9 @@ struct ContextSharedPart Context::ConfigReloadCallback config_reload_callback; ContextSharedPart() - : access_control(std::make_unique()), macros(std::make_unique()) + : access_control(std::make_unique()) + , global_overcommit_tracker(&process_list) + , macros(std::make_unique()) { /// TODO: make it singleton (?) static std::atomic num_calls{0}; @@ -467,6 +470,7 @@ std::unique_lock Context::getLock() const ProcessList & Context::getProcessList() { return shared->process_list; } const ProcessList & Context::getProcessList() const { return shared->process_list; } +OvercommitTracker * Context::getGlobalOvercommitTracker() const { return &shared->global_overcommit_tracker; } MergeList & Context::getMergeList() { return shared->merge_list; } const MergeList & Context::getMergeList() const { return shared->merge_list; } ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 7d31a8375d8..2ddd71d1800 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -28,6 +28,7 @@ namespace Poco::Net { class IPAddress; } namespace zkutil { class ZooKeeper; } +struct OvercommitTracker; namespace DB { @@ -650,6 +651,8 @@ public: ProcessList & getProcessList(); const ProcessList & getProcessList() const; + OvercommitTracker * getGlobalOvercommitTracker() const; + MergeList & getMergeList(); const MergeList & getMergeList() const; diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 81afa990d3b..d2f34bd998c 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -179,12 +179,16 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as res = std::make_shared(*this, process_it); ProcessListForUser & user_process_list = user_to_queries[client_info.current_user]; - user_process_list.queries.emplace(client_info.current_query_id, &res->get()); + { + BlockQueryIfMemoryLimit block_query{user_process_list.user_overcommit_tracker}; + user_process_list.queries.emplace(client_info.current_query_id, &res->get()); + } process_it->setUserProcessList(&user_process_list); /// Track memory usage for all simultaneously running queries from single user. user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user); + user_process_list.user_memory_tracker.setSoftLimit(settings.max_guaranteed_memory_usage_for_user); user_process_list.user_memory_tracker.setDescription("(for user)"); /// Actualize thread group info @@ -198,6 +202,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as /// Set query-level memory trackers thread_group->memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage); + thread_group->memory_tracker.setSoftLimit(settings.max_guaranteed_memory_usage); if (query_context->hasTraceCollector()) { @@ -261,6 +266,7 @@ ProcessListEntry::~ProcessListEntry() { if (running_query->second == process_list_element_ptr) { + BlockQueryIfMemoryLimit block_query{user_process_list.user_overcommit_tracker}; user_process_list.queries.erase(running_query->first); found = true; } @@ -434,7 +440,11 @@ ProcessList::Info ProcessList::getInfo(bool get_thread_list, bool get_profile_ev } -ProcessListForUser::ProcessListForUser() = default; +ProcessListForUser::ProcessListForUser() + : user_overcommit_tracker(this) +{ + user_memory_tracker.setOvercommitTracker(&user_overcommit_tracker); +} ProcessListForUserInfo ProcessListForUser::getInfo(bool get_profile_events) const diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 02be24bb2bd..e5a2e4f280d 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -146,6 +147,11 @@ public: ThrottlerPtr getUserNetworkThrottler(); + MemoryTracker * getMemoryTracker() const + { + return &thread_group->memory_tracker; + } + bool updateProgressIn(const Progress & value) { CurrentThread::updateProgressIn(value); @@ -203,6 +209,8 @@ struct ProcessListForUser /// Limit and counter for memory of all simultaneously running queries of single user. MemoryTracker user_memory_tracker{VariableContext::User}; + UserOvercommitTracker user_overcommit_tracker; + /// Count network usage for all simultaneously running queries of single user. ThrottlerPtr user_throttler; @@ -308,6 +316,14 @@ public: max_size = max_size_; } + template + void processEachQueryStatus(F && func) const + { + std::lock_guard lk(mutex); + for (auto && query : processes) + func(query); + } + /// Try call cancel() for input and output streams of query with specified id and user CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill = false); From 9464dc2fd31f6b117e59639f7b55f4f35f5c1734 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 22 Oct 2021 18:15:33 +0300 Subject: [PATCH 002/164] Add waiting timeout --- programs/server/Server.cpp | 6 +++++- src/Common/OvercommitTracker.cpp | 17 +++++++++++++++-- src/Common/OvercommitTracker.h | 12 ++++++++---- src/Core/Settings.h | 2 ++ src/Interpreters/ProcessList.cpp | 1 + 5 files changed, 31 insertions(+), 7 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 7be5aaea429..d23a604e757 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -841,7 +841,11 @@ if (ThreadFuzzer::instance().isEffective()) total_memory_tracker.setHardLimit(max_server_memory_usage); total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - total_memory_tracker.setOvercommitTracker(global_context->getGlobalOvercommitTracker()); + + UInt64 max_overcommit_wait_time = config->getUInt64("memory_usage_overcommit_max_wait_microseconds", 0); + auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); + global_overcommit_tracker->setMaxWaitTime(max_overcommit_wait_time); + total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); // FIXME logging-related things need synchronization -- see the 'Logger * log' saved // in a lot of places. For now, disable updating log configuration without server restart. diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index a2c5ae0ba4b..ae7720b878c 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -1,7 +1,21 @@ #include "OvercommitTracker.h" +#include #include +using namespace std::chrono_literals; + +OvercommitTracker::OvercommitTracker() + : max_wait_time(0us) + , picked_tracker(nullptr) + , cancelation_state(QueryCancelationState::NONE) +{} + +void OvercommitTracker::setMaxWaitTime(UInt64 wait_time) +{ + max_wait_time = wait_time * 1us; +} + bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker) { std::unique_lock lk(overcommit_m); @@ -11,8 +25,7 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker) if (tracker == picked_tracker) return true; - auto now = std::chrono::system_clock::now(); - return cv.wait_until(lk, now, [this]() + return cv.wait_for(lk, max_wait_time, [this]() { return cancelation_state == QueryCancelationState::NONE; }); diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index dcf6a9262ab..cacda0a5315 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -27,7 +28,9 @@ class MemoryTracker; struct OvercommitTracker : boost::noncopyable { - OvercommitTracker() = default; + OvercommitTracker(); + + void setMaxWaitTime(UInt64 wait_time); bool needToStopQuery(MemoryTracker * tracker); @@ -50,6 +53,8 @@ protected: mutable std::mutex overcommit_m; mutable std::condition_variable cv; + std::chrono::microseconds max_wait_time; + enum class QueryCancelationState { NONE, @@ -57,7 +62,7 @@ protected: }; MemoryTracker * picked_tracker; - QueryCancelationState cancelation_state = QueryCancelationState::NONE; + QueryCancelationState cancelation_state; private: @@ -115,8 +120,7 @@ struct BlockQueryIfMemoryLimit { if (overcommit_tracker.cancelation_state == OvercommitTracker::QueryCancelationState::RUNNING) { - //TODO: Add timeout - overcommit_tracker.cv.wait(lk, [&overcommit_tracker]() + overcommit_tracker.cv.wait_for(lk, overcommit_tracker.max_wait_time, [&overcommit_tracker]() { return overcommit_tracker.cancelation_state == OvercommitTracker::QueryCancelationState::NONE; }); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 906f10ae653..f219a03d47f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -356,6 +356,8 @@ class IColumn; M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ \ + M(UInt64, memory_usage_overcommit_max_wait_microseconds, 0, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown", 0) \ + \ M(UInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \ M(UInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \ M(UInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.", 0)\ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index d2f34bd998c..1719f683add 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -190,6 +190,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user); user_process_list.user_memory_tracker.setSoftLimit(settings.max_guaranteed_memory_usage_for_user); user_process_list.user_memory_tracker.setDescription("(for user)"); + user_process_list.user_overcommit_tracker.setMaxWaitTime(settings.memory_usage_overcommit_max_wait_microseconds); /// Actualize thread group info if (auto thread_group = CurrentThread::getGroup()) From 9c7e6c78145f6552e664fb60862e220825ed489e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 26 Oct 2021 15:32:17 +0300 Subject: [PATCH 003/164] Use per user soft limit in GlobalOvercommitTracker --- src/Common/MemoryTracker.cpp | 6 ++++++ src/Common/MemoryTracker.h | 6 ++++++ src/Common/OvercommitTracker.cpp | 8 ++++++-- src/Interpreters/ProcessList.h | 1 + 4 files changed, 19 insertions(+), 2 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 61aa470e2d4..4c0e6fa7503 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -365,6 +365,12 @@ OvercommitRatio MemoryTracker::getOvercommitRatio() } +OvercommitRatio MemoryTracker::getOvercommitRatio(Int64 limit) +{ + return { amount.load(std::memory_order_relaxed), limit }; +} + + void MemoryTracker::resetCounters() { amount.store(0, std::memory_order_relaxed); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index e055a347617..7602a15e2a4 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -114,6 +114,11 @@ public: void setSoftLimit(Int64 value); void setHardLimit(Int64 value); + Int64 getSoftLimit() const + { + return soft_limit.load(std::memory_order_relaxed); + } + /** Set limit if it was not set. * Otherwise, set limit to new value, if new value is greater than previous limit. */ @@ -164,6 +169,7 @@ public: } OvercommitRatio getOvercommitRatio(); + OvercommitRatio getOvercommitRatio(Int64 limit); void setOvercommitTracker(OvercommitTracker * tracker) noexcept { diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index ae7720b878c..56b7f40ef6f 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -39,7 +39,7 @@ void UserOvercommitTracker::pickQueryToExcludeImpl() { MemoryTracker * current_tracker = nullptr; OvercommitRatio current_ratio{0, 0}; - //TODO: ensure this container is not being modified + // At this moment query list must be read only for (auto const & query : user_process_list->queries) { auto * memory_tracker = query.second->getMemoryTracker(); @@ -61,7 +61,11 @@ void GlobalOvercommitTracker::pickQueryToExcludeImpl() process_list->processEachQueryStatus([&](DB::QueryStatus const & query) { auto * memory_tracker = query.getMemoryTracker(); - auto ratio = memory_tracker->getOvercommitRatio(); + Int64 user_soft_limit = 0; + if (auto const * user_process_list = query.getUserProcessList()) + user_soft_limit = user_process_list->user_memory_tracker.getSoftLimit(); + + auto ratio = memory_tracker->getOvercommitRatio(user_soft_limit); if (current_ratio < ratio) { current_tracker = memory_tracker; diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index e5a2e4f280d..328467ee83b 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -80,6 +80,7 @@ protected: friend class ThreadStatus; friend class CurrentThread; friend class ProcessListEntry; + friend struct ::GlobalOvercommitTracker; String query; ClientInfo client_info; From 32904701f92932bc7e47481bef815da3d3c74de8 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 26 Oct 2021 16:21:58 +0300 Subject: [PATCH 004/164] Fix OvercommitTracker when soft limit is not set --- src/Common/OvercommitTracker.cpp | 22 +++++++++++++++++++--- src/Common/OvercommitTracker.h | 19 +++++++++---------- 2 files changed, 28 insertions(+), 13 deletions(-) diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index 56b7f40ef6f..21e27b7c825 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -22,8 +22,11 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker) pickQueryToExclude(); assert(cancelation_state == QueryCancelationState::RUNNING); - if (tracker == picked_tracker) + if (picked_tracker == tracker || picked_tracker == nullptr) + { + ++waiting_to_stop; return true; + } return cv.wait_for(lk, max_wait_time, [this]() { @@ -31,6 +34,21 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker) }); } +void OvercommitTracker::unsubscribe(MemoryTracker * tracker) +{ + std::unique_lock lk(overcommit_m); + if (picked_tracker == tracker || picked_tracker == nullptr) + { + --waiting_to_stop; + if (waiting_to_stop == 0) + { + picked_tracker = nullptr; + cancelation_state = QueryCancelationState::NONE; + cv.notify_all(); + } + } +} + UserOvercommitTracker::UserOvercommitTracker(DB::ProcessListForUser * user_process_list_) : user_process_list(user_process_list_) {} @@ -50,7 +68,6 @@ void UserOvercommitTracker::pickQueryToExcludeImpl() current_ratio = ratio; } } - assert(current_tracker != nullptr); picked_tracker = current_tracker; } @@ -72,6 +89,5 @@ void GlobalOvercommitTracker::pickQueryToExcludeImpl() current_ratio = ratio; } }); - assert(current_tracker != nullptr); picked_tracker = current_tracker; } diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index cacda0a5315..d2e20cf8458 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -34,16 +34,7 @@ struct OvercommitTracker : boost::noncopyable bool needToStopQuery(MemoryTracker * tracker); - void unsubscribe(MemoryTracker * tracker) - { - std::unique_lock lk(overcommit_m); - if (tracker == picked_tracker) - { - picked_tracker = nullptr; - cancelation_state = QueryCancelationState::NONE; - cv.notify_all(); - } - } + void unsubscribe(MemoryTracker * tracker); virtual ~OvercommitTracker() = default; @@ -61,11 +52,19 @@ protected: RUNNING, }; + // Specifies memory tracker of the choosen to stop query. + // If soft limit is not set, all the queries which reach hard limit must stop. + // This case is represented as picked tracker pointer is set to nullptr and + // overcommit tracker is in RUNNING state. MemoryTracker * picked_tracker; QueryCancelationState cancelation_state; private: + // Number of queries are being canceled at the moment. Overcommit tracker + // must be in RUNNING state until this counter is not equal to 0. + UInt64 waiting_to_stop = 0; + void pickQueryToExclude() { if (cancelation_state != QueryCancelationState::RUNNING) From 87960c3f3dd1e129167f314119ed2de7afe7a53d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 26 Oct 2021 16:55:57 +0300 Subject: [PATCH 005/164] Fix division by zero --- src/Common/OvercommitTracker.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index d2e20cf8458..5779f5ba392 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -17,7 +17,8 @@ struct OvercommitRatio friend bool operator<(OvercommitRatio const& lhs, OvercommitRatio const& rhs) noexcept { - return (lhs.commited / lhs.soft_limit) < (rhs.commited / rhs.soft_limit); + // (a / b < c / d) <=> (a * d < c * b) + return (lhs.commited * rhs.soft_limit) < (rhs.commited * lhs.soft_limit); } Int64 commited; From cbe6d89c69d5254da4afaa6f31e5ef52863e2e84 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 9 Nov 2021 16:40:23 +0300 Subject: [PATCH 006/164] Fix typos --- src/Common/OvercommitTracker.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 5779f5ba392..2de24623b1e 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -10,18 +10,18 @@ struct OvercommitRatio { - OvercommitRatio(Int64 commited_, Int64 soft_limit_) - : commited(commited_) + OvercommitRatio(Int64 committed_, Int64 soft_limit_) + : committed(committed_) , soft_limit(soft_limit_) {} friend bool operator<(OvercommitRatio const& lhs, OvercommitRatio const& rhs) noexcept { // (a / b < c / d) <=> (a * d < c * b) - return (lhs.commited * rhs.soft_limit) < (rhs.commited * lhs.soft_limit); + return (lhs.committed * rhs.soft_limit) < (rhs.committed * lhs.soft_limit); } - Int64 commited; + Int64 committed; Int64 soft_limit; }; @@ -53,7 +53,7 @@ protected: RUNNING, }; - // Specifies memory tracker of the choosen to stop query. + // Specifies memory tracker of the chosen to stop query. // If soft limit is not set, all the queries which reach hard limit must stop. // This case is represented as picked tracker pointer is set to nullptr and // overcommit tracker is in RUNNING state. From 12101d82aab42a2b5e8d44ddf52e690930d90180 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 6 Dec 2021 21:34:52 +0300 Subject: [PATCH 007/164] Fix overcommit ratio comparison and race condition --- src/Common/MemoryTracker.cpp | 23 +++------ src/Common/MemoryTracker.h | 2 +- src/Common/OvercommitTracker.cpp | 37 +++++++++----- src/Common/OvercommitTracker.h | 8 ++- src/Interpreters/ProcessList.cpp | 13 +++-- .../0_stateless/02104_overcommit_memory.sh | 50 +++++++++++++++++++ 6 files changed, 95 insertions(+), 38 deletions(-) create mode 100755 tests/queries/0_stateless/02104_overcommit_memory.sh diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 4c0e6fa7503..d93b13ebc27 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -101,17 +101,6 @@ MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : MemoryTracker::~MemoryTracker() { - if (level == VariableContext::Process) - { - auto * loaded_next = getParent(); - while (loaded_next != nullptr) - { - if (auto * next_overcommit_tracker = loaded_next->overcommit_tracker) - next_overcommit_tracker->unsubscribe(this); - loaded_next = loaded_next->getParent(); - } - } - if ((level == VariableContext::Process || level == VariableContext::User) && peak) { try @@ -141,7 +130,7 @@ void MemoryTracker::logMemoryUsage(Int64 current) const } -void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) +void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker) { if (size < 0) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Negative size ({}) is passed to MemoryTracker. It is a bug.", size); @@ -150,7 +139,8 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) { /// Since the BlockerInThread should respect the level, we should go to the next parent. if (auto * loaded_next = parent.load(std::memory_order_relaxed)) - loaded_next->allocImpl(size, throw_if_memory_exceeded); + loaded_next->allocImpl(size, throw_if_memory_exceeded, + level == VariableContext::Process ? this : query_tracker); return; } @@ -233,8 +223,8 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { bool need_to_throw = true; - if (overcommit_tracker) - need_to_throw = overcommit_tracker->needToStopQuery(this); + if (!!overcommit_tracker && !!query_tracker) + need_to_throw = overcommit_tracker->needToStopQuery(query_tracker); if (need_to_throw) { @@ -278,7 +268,8 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) } if (auto * loaded_next = parent.load(std::memory_order_relaxed)) - loaded_next->allocImpl(size, throw_if_memory_exceeded); + loaded_next->allocImpl(size, throw_if_memory_exceeded, + level == VariableContext::Process ? this : query_tracker); } void MemoryTracker::alloc(Int64 size) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 7602a15e2a4..85139fcc386 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -86,7 +86,7 @@ public: void allocNoThrow(Int64 size); - void allocImpl(Int64 size, bool throw_if_memory_exceeded); + void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); void realloc(Int64 old_size, Int64 new_size) { diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index 21e27b7c825..9e3917981f9 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -1,5 +1,6 @@ #include "OvercommitTracker.h" +#include #include #include @@ -22,12 +23,14 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker) pickQueryToExclude(); assert(cancelation_state == QueryCancelationState::RUNNING); - if (picked_tracker == tracker || picked_tracker == nullptr) + + if (picked_tracker == nullptr) { - ++waiting_to_stop; + cancelation_state = QueryCancelationState::NONE; return true; } - + if (picked_tracker == tracker) + return true; return cv.wait_for(lk, max_wait_time, [this]() { return cancelation_state == QueryCancelationState::NONE; @@ -37,15 +40,13 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker) void OvercommitTracker::unsubscribe(MemoryTracker * tracker) { std::unique_lock lk(overcommit_m); - if (picked_tracker == tracker || picked_tracker == nullptr) + if (picked_tracker == tracker) { - --waiting_to_stop; - if (waiting_to_stop == 0) - { - picked_tracker = nullptr; - cancelation_state = QueryCancelationState::NONE; - cv.notify_all(); - } + LOG_DEBUG(&Poco::Logger::get("OvercommitTracker"), "Picked query stopped"); + + picked_tracker = nullptr; + cancelation_state = QueryCancelationState::NONE; + cv.notify_all(); } } @@ -58,16 +59,26 @@ void UserOvercommitTracker::pickQueryToExcludeImpl() MemoryTracker * current_tracker = nullptr; OvercommitRatio current_ratio{0, 0}; // At this moment query list must be read only - for (auto const & query : user_process_list->queries) + auto & queries = user_process_list->queries; + LOG_DEBUG(&Poco::Logger::get("OvercommitTracker"), + "Trying to choose query to stop from {} queries", queries.size()); + for (auto const & query : queries) { + if (query.second->isKilled()) + continue; auto * memory_tracker = query.second->getMemoryTracker(); auto ratio = memory_tracker->getOvercommitRatio(); - if (current_ratio < ratio) + LOG_DEBUG(&Poco::Logger::get("OvercommitTracker"), + "Query has ratio {}/{}", ratio.committed, ratio.soft_limit); + if (ratio.soft_limit != 0 && current_ratio < ratio) { current_tracker = memory_tracker; current_ratio = ratio; } } + LOG_DEBUG(&Poco::Logger::get("OvercommitTracker"), + "Selected to stop query with overcommit ratio {}/{}", + current_ratio.committed, current_ratio.soft_limit); picked_tracker = current_tracker; } diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 2de24623b1e..5d02d47259c 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -18,7 +18,9 @@ struct OvercommitRatio friend bool operator<(OvercommitRatio const& lhs, OvercommitRatio const& rhs) noexcept { // (a / b < c / d) <=> (a * d < c * b) - return (lhs.committed * rhs.soft_limit) < (rhs.committed * lhs.soft_limit); + return (lhs.committed * rhs.soft_limit) < (rhs.committed * lhs.soft_limit) + || (lhs.soft_limit == 0 && rhs.soft_limit > 0) + || (lhs.committed == 0 && rhs.committed == 0 && lhs.soft_limit > rhs.soft_limit); } Int64 committed; @@ -62,10 +64,6 @@ protected: private: - // Number of queries are being canceled at the moment. Overcommit tracker - // must be in RUNNING state until this counter is not equal to 0. - UInt64 waiting_to_stop = 0; - void pickQueryToExclude() { if (cancelation_state != QueryCancelationState::RUNNING) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 1719f683add..5ffc65762aa 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -7,6 +7,7 @@ #include #include #include +#include "Common/tests/gtest_global_context.h" #include #include #include @@ -249,9 +250,6 @@ ProcessListEntry::~ProcessListEntry() const QueryStatus * process_list_element_ptr = &*it; - /// This removes the memory_tracker of one request. - parent.processes.erase(it); - auto user_process_list_it = parent.user_to_queries.find(user); if (user_process_list_it == parent.user_to_queries.end()) { @@ -273,6 +271,9 @@ ProcessListEntry::~ProcessListEntry() } } + /// This removes the memory_tracker of one request. + parent.processes.erase(it); + if (!found) { LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); @@ -303,6 +304,12 @@ QueryStatus::QueryStatus( QueryStatus::~QueryStatus() { assert(executors.empty()); + + auto * memory_tracker = getMemoryTracker(); + if (user_process_list) + user_process_list->user_overcommit_tracker.unsubscribe(memory_tracker); + if (auto context = getContext()) + context->getGlobalOvercommitTracker()->unsubscribe(memory_tracker); } CancellationCode QueryStatus::cancelQuery(bool) diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh new file mode 100755 index 00000000000..77e8758c18f --- /dev/null +++ b/tests/queries/0_stateless/02104_overcommit_memory.sh @@ -0,0 +1,50 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q 'SELECT number FROM numbers(1000000) GROUP BY number SETTINGS max_memory_usage_for_user = 1' 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q 'SELECT number FROM numbers(1000000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user = 1' 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'FAIL' || echo 'OK' + +$CLICKHOUSE_CLIENT -q 'CREATE USER IF NOT EXISTS u1 IDENTIFIED WITH no_password' +$CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u1' + +function overcommited() +{ + while true; do + $CLICKHOUSE_CLIENT -u u1 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_guaranteed_memory_usage=1,memory_usage_overcommit_max_wait_microseconds=500' 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo "OVERCOMMITED WITH USER LIMIT IS KILLED" + done +} + +function expect_execution() +{ + while true; do + $CLICKHOUSE_CLIENT -u u1 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_memory_usage_for_user=5000000,max_guaranteed_memory_usage=2,memory_usage_overcommit_max_wait_microseconds=500' >/dev/null 2>/dev/null 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo "bla\n" + done +} + +export -f overcommited +export -f expect_execution + +function user_test() +{ + for _ in {1..10}; + do + timeout 3 bash -c overcommited & + timeout 3 bash -c expect_execution & + done; + + wait +} + +output=$(user_test) + +if test -z "$output" +then + echo "OVERCOMMITED WITH USER LIMIT WAS NOT KILLED" +else + echo "OVERCOMMITED WITH USER LIMIT WAS KILLED" +fi + +$CLICKHOUSE_CLIENT -q 'DROP USER IF EXISTS u1' From 1465d4c0adf45e7f78080338ca6fe39fb6d10804 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 6 Dec 2021 21:47:18 +0300 Subject: [PATCH 008/164] Add reference file --- tests/queries/0_stateless/02104_overcommit_memory.reference | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02104_overcommit_memory.reference diff --git a/tests/queries/0_stateless/02104_overcommit_memory.reference b/tests/queries/0_stateless/02104_overcommit_memory.reference new file mode 100644 index 00000000000..5e7557b5f74 --- /dev/null +++ b/tests/queries/0_stateless/02104_overcommit_memory.reference @@ -0,0 +1,3 @@ +OK +OK +OVERCOMMITED WITH USER LIMIT WAS KILLED From de90a9ef71db9e0b7a7bfeb47eb6a5aa27c175f0 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 6 Dec 2021 21:51:46 +0300 Subject: [PATCH 009/164] Update test --- tests/queries/0_stateless/02104_overcommit_memory.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh index 77e8758c18f..05ff6751f4d 100755 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ b/tests/queries/0_stateless/02104_overcommit_memory.sh @@ -20,7 +20,7 @@ function overcommited() function expect_execution() { while true; do - $CLICKHOUSE_CLIENT -u u1 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_memory_usage_for_user=5000000,max_guaranteed_memory_usage=2,memory_usage_overcommit_max_wait_microseconds=500' >/dev/null 2>/dev/null 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo "bla\n" + $CLICKHOUSE_CLIENT -u u1 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_memory_usage_for_user=5000000,max_guaranteed_memory_usage=2,memory_usage_overcommit_max_wait_microseconds=500' >/dev/null 2>/dev/null done } From e10c595ef38db07aa3d2279142286ecba44c29c0 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 6 Dec 2021 22:42:51 +0300 Subject: [PATCH 010/164] Make test not flaky --- tests/queries/0_stateless/02104_overcommit_memory.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh index 05ff6751f4d..2d33d8897f2 100755 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ b/tests/queries/0_stateless/02104_overcommit_memory.sh @@ -4,8 +4,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q 'SELECT number FROM numbers(1000000) GROUP BY number SETTINGS max_memory_usage_for_user = 1' 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -q 'SELECT number FROM numbers(1000000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user = 1' 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'FAIL' || echo 'OK' +$CLICKHOUSE_CLIENT -q 'SELECT number FROM numbers(10000000) GROUP BY number SETTINGS max_memory_usage_for_user = 1' 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q 'SELECT number FROM numbers(10000000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user = 1' 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'FAIL' || echo 'OK' $CLICKHOUSE_CLIENT -q 'CREATE USER IF NOT EXISTS u1 IDENTIFIED WITH no_password' $CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u1' From 5639fc35d2e9582e543625fe1ef5b1d8d90eb749 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 6 Dec 2021 23:26:33 +0300 Subject: [PATCH 011/164] remove queries --- tests/queries/0_stateless/02104_overcommit_memory.reference | 2 -- tests/queries/0_stateless/02104_overcommit_memory.sh | 3 --- 2 files changed, 5 deletions(-) diff --git a/tests/queries/0_stateless/02104_overcommit_memory.reference b/tests/queries/0_stateless/02104_overcommit_memory.reference index 5e7557b5f74..b108f48e0fa 100644 --- a/tests/queries/0_stateless/02104_overcommit_memory.reference +++ b/tests/queries/0_stateless/02104_overcommit_memory.reference @@ -1,3 +1 @@ -OK -OK OVERCOMMITED WITH USER LIMIT WAS KILLED diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh index 2d33d8897f2..e49ec5e2335 100755 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ b/tests/queries/0_stateless/02104_overcommit_memory.sh @@ -4,9 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q 'SELECT number FROM numbers(10000000) GROUP BY number SETTINGS max_memory_usage_for_user = 1' 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -q 'SELECT number FROM numbers(10000000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user = 1' 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'FAIL' || echo 'OK' - $CLICKHOUSE_CLIENT -q 'CREATE USER IF NOT EXISTS u1 IDENTIFIED WITH no_password' $CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u1' From 697ab52387b6697c3b8b9d7ebf0931f320778055 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 7 Dec 2021 01:06:00 +0300 Subject: [PATCH 012/164] Add logs to GlobalOvercommitTracker --- src/Common/OvercommitTracker.cpp | 21 +++++++++++++-------- src/Common/OvercommitTracker.h | 3 +++ 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index 9e3917981f9..99424238dc9 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -1,6 +1,5 @@ #include "OvercommitTracker.h" -#include #include #include @@ -60,24 +59,21 @@ void UserOvercommitTracker::pickQueryToExcludeImpl() OvercommitRatio current_ratio{0, 0}; // At this moment query list must be read only auto & queries = user_process_list->queries; - LOG_DEBUG(&Poco::Logger::get("OvercommitTracker"), - "Trying to choose query to stop from {} queries", queries.size()); + LOG_DEBUG(logger, "Trying to choose query to stop from {} queries", queries.size()); for (auto const & query : queries) { if (query.second->isKilled()) continue; auto * memory_tracker = query.second->getMemoryTracker(); auto ratio = memory_tracker->getOvercommitRatio(); - LOG_DEBUG(&Poco::Logger::get("OvercommitTracker"), - "Query has ratio {}/{}", ratio.committed, ratio.soft_limit); + LOG_DEBUG(logger, "Query has ratio {}/{}", ratio.committed, ratio.soft_limit); if (ratio.soft_limit != 0 && current_ratio < ratio) { current_tracker = memory_tracker; current_ratio = ratio; } } - LOG_DEBUG(&Poco::Logger::get("OvercommitTracker"), - "Selected to stop query with overcommit ratio {}/{}", + LOG_DEBUG(logger, "Selected to stop query with overcommit ratio {}/{}", current_ratio.committed, current_ratio.soft_limit); picked_tracker = current_tracker; } @@ -86,19 +82,28 @@ void GlobalOvercommitTracker::pickQueryToExcludeImpl() { MemoryTracker * current_tracker = nullptr; OvercommitRatio current_ratio{0, 0}; + LOG_DEBUG(logger, "Trying to choose query to stop from {} queries", process_list->size()); process_list->processEachQueryStatus([&](DB::QueryStatus const & query) { - auto * memory_tracker = query.getMemoryTracker(); + if (query.isKilled()) + return; + Int64 user_soft_limit = 0; if (auto const * user_process_list = query.getUserProcessList()) user_soft_limit = user_process_list->user_memory_tracker.getSoftLimit(); + if (user_soft_limit == 0) + return; + auto * memory_tracker = query.getMemoryTracker(); auto ratio = memory_tracker->getOvercommitRatio(user_soft_limit); + LOG_DEBUG(logger, "Query has ratio {}/{}", ratio.committed, ratio.soft_limit); if (current_ratio < ratio) { current_tracker = memory_tracker; current_ratio = ratio; } }); + LOG_DEBUG(logger, "Selected to stop query with overcommit ratio {}/{}", + current_ratio.committed, current_ratio.soft_limit); picked_tracker = current_tracker; } diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 5d02d47259c..845402c9a64 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -93,6 +94,7 @@ protected: private: DB::ProcessListForUser * user_process_list; + Poco::Logger * logger = &Poco::Logger::get("UserOvercommitTracker"); }; struct GlobalOvercommitTracker : OvercommitTracker @@ -108,6 +110,7 @@ protected: private: DB::ProcessList * process_list; + Poco::Logger * logger = &Poco::Logger::get("GlobalOvercommitTracker"); }; struct BlockQueryIfMemoryLimit From 366b105c2750a5cfc01b36bf12e442427dd27029 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 7 Dec 2021 01:16:09 +0300 Subject: [PATCH 013/164] Fix data race --- src/Common/OvercommitTracker.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index 99424238dc9..4731741057c 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -1,6 +1,7 @@ #include "OvercommitTracker.h" #include +#include #include using namespace std::chrono_literals; @@ -13,6 +14,7 @@ OvercommitTracker::OvercommitTracker() void OvercommitTracker::setMaxWaitTime(UInt64 wait_time) { + std::unique_lock lk(overcommit_m); max_wait_time = wait_time * 1us; } From 3ba91b7049b0441b78732394fd251a909826fa97 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 7 Dec 2021 12:58:08 +0300 Subject: [PATCH 014/164] Increase timeout in test --- tests/queries/0_stateless/02104_overcommit_memory.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh index e49ec5e2335..b4fedff9909 100755 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ b/tests/queries/0_stateless/02104_overcommit_memory.sh @@ -28,8 +28,8 @@ function user_test() { for _ in {1..10}; do - timeout 3 bash -c overcommited & - timeout 3 bash -c expect_execution & + timeout 10 bash -c overcommited & + timeout 10 bash -c expect_execution & done; wait From a956a468f77ff89768579eeb21ba487b7844b573 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 7 Dec 2021 13:03:51 +0300 Subject: [PATCH 015/164] Remove log entry with data race --- src/Common/OvercommitTracker.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index 4731741057c..63cd069fa15 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -84,7 +84,6 @@ void GlobalOvercommitTracker::pickQueryToExcludeImpl() { MemoryTracker * current_tracker = nullptr; OvercommitRatio current_ratio{0, 0}; - LOG_DEBUG(logger, "Trying to choose query to stop from {} queries", process_list->size()); process_list->processEachQueryStatus([&](DB::QueryStatus const & query) { if (query.isKilled()) From 8d8222acf24a037c6a99c065b801078716d43bff Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 7 Dec 2021 13:11:31 +0300 Subject: [PATCH 016/164] Fix build with gcc --- src/Interpreters/ProcessList.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 61a9233429a..f27406c92b4 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -312,8 +312,8 @@ QueryStatus::~QueryStatus() auto * memory_tracker = getMemoryTracker(); if (user_process_list) user_process_list->user_overcommit_tracker.unsubscribe(memory_tracker); - if (auto context = getContext()) - context->getGlobalOvercommitTracker()->unsubscribe(memory_tracker); + if (auto shared_context = getContext()) + shared_context->getGlobalOvercommitTracker()->unsubscribe(memory_tracker); } CancellationCode QueryStatus::cancelQuery(bool) From 03737a32df058cc307c3041bb77fc74ad85a56f9 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 13 Dec 2021 01:34:23 +0300 Subject: [PATCH 017/164] Make test non-parallel --- tests/queries/0_stateless/02104_overcommit_memory.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh index b4fedff9909..140557304c6 100755 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ b/tests/queries/0_stateless/02104_overcommit_memory.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From de432f927037da48059a0a86353f65cef1795484 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 14 Dec 2021 16:39:53 +0300 Subject: [PATCH 018/164] Fix possible NPE when thread_group is not set --- src/Common/OvercommitTracker.cpp | 6 ++++++ src/Interpreters/ProcessList.cpp | 12 +++++++----- src/Interpreters/ProcessList.h | 2 ++ 3 files changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index 63cd069fa15..7ac74e761db 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -66,7 +66,11 @@ void UserOvercommitTracker::pickQueryToExcludeImpl() { if (query.second->isKilled()) continue; + auto * memory_tracker = query.second->getMemoryTracker(); + if (!memory_tracker) + continue; + auto ratio = memory_tracker->getOvercommitRatio(); LOG_DEBUG(logger, "Query has ratio {}/{}", ratio.committed, ratio.soft_limit); if (ratio.soft_limit != 0 && current_ratio < ratio) @@ -96,6 +100,8 @@ void GlobalOvercommitTracker::pickQueryToExcludeImpl() return; auto * memory_tracker = query.getMemoryTracker(); + if (!memory_tracker) + return; auto ratio = memory_tracker->getOvercommitRatio(user_soft_limit); LOG_DEBUG(logger, "Query has ratio {}/{}", ratio.committed, ratio.soft_limit); if (current_ratio < ratio) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index f27406c92b4..a5465223b51 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -309,11 +309,13 @@ QueryStatus::~QueryStatus() { assert(executors.empty()); - auto * memory_tracker = getMemoryTracker(); - if (user_process_list) - user_process_list->user_overcommit_tracker.unsubscribe(memory_tracker); - if (auto shared_context = getContext()) - shared_context->getGlobalOvercommitTracker()->unsubscribe(memory_tracker); + if (auto * memory_tracker = getMemoryTracker()) + { + if (user_process_list) + user_process_list->user_overcommit_tracker.unsubscribe(memory_tracker); + if (auto shared_context = getContext()) + shared_context->getGlobalOvercommitTracker()->unsubscribe(memory_tracker); + } } CancellationCode QueryStatus::cancelQuery(bool) diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index ec3bb93b8d9..b1a9da314c6 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -149,6 +149,8 @@ public: MemoryTracker * getMemoryTracker() const { + if (!thread_group) + return nullptr; return &thread_group->memory_tracker; } From 0d91864ec02c1728b0771768d546cf1baba8371f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 15 Dec 2021 18:05:46 +0300 Subject: [PATCH 019/164] Avoid possible data race when thread_group is initialized --- src/Interpreters/ProcessList.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index a5465223b51..1cbaf9f3ad1 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -182,11 +182,6 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as res = std::make_shared(*this, process_it); ProcessListForUser & user_process_list = user_to_queries[client_info.current_user]; - { - BlockQueryIfMemoryLimit block_query{user_process_list.user_overcommit_tracker}; - user_process_list.queries.emplace(client_info.current_query_id, &res->get()); - } - process_it->setUserProcessList(&user_process_list); /// Track memory usage for all simultaneously running queries from single user. @@ -225,6 +220,11 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as process_it->thread_group = std::move(thread_group); } + { + BlockQueryIfMemoryLimit block_query{user_process_list.user_overcommit_tracker}; + user_process_list.queries.emplace(client_info.current_query_id, &res->get()); + } + if (!user_process_list.user_throttler) { if (settings.max_network_bandwidth_for_user) From dce9390eceae84b8af7416ebbb3ae5c1c6a4288c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 16 Dec 2021 16:46:15 +0300 Subject: [PATCH 020/164] Init thread_group before QueryStatus creation --- src/Interpreters/ProcessList.cpp | 41 ++++++++++++++++++-------------- src/Interpreters/ProcessList.h | 3 ++- 2 files changed, 25 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 1cbaf9f3ad1..9d6e7b61acc 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -176,28 +176,17 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING); } - auto process_it = processes.emplace(processes.end(), - query_context, query_, client_info, priorities.insert(settings.priority)); - - res = std::make_shared(*this, process_it); - ProcessListForUser & user_process_list = user_to_queries[client_info.current_user]; - process_it->setUserProcessList(&user_process_list); - - /// Track memory usage for all simultaneously running queries from single user. - user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user); - user_process_list.user_memory_tracker.setSoftLimit(settings.max_guaranteed_memory_usage_for_user); - user_process_list.user_memory_tracker.setDescription("(for user)"); - user_process_list.user_overcommit_tracker.setMaxWaitTime(settings.memory_usage_overcommit_max_wait_microseconds); /// Actualize thread group info - if (auto thread_group = CurrentThread::getGroup()) + auto thread_group = CurrentThread::getGroup(); + if (thread_group) { std::lock_guard lock_thread_group(thread_group->mutex); thread_group->performance_counters.setParent(&user_process_list.user_performance_counters); thread_group->memory_tracker.setParent(&user_process_list.user_memory_tracker); - thread_group->query = process_it->query; - thread_group->normalized_query_hash = normalizedQueryHash(process_it->query); + thread_group->query = query_; + thread_group->normalized_query_hash = normalizedQueryHash(query_); /// Set query-level memory trackers thread_group->memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage); @@ -216,15 +205,26 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as /// NOTE: Do not set the limit for thread-level memory tracker since it could show unreal values /// since allocation and deallocation could happen in different threads - - process_it->thread_group = std::move(thread_group); } + auto process_it = processes.emplace(processes.end(), + query_context, query_, client_info, priorities.insert(settings.priority), std::move(thread_group)); + + res = std::make_shared(*this, process_it); + + process_it->setUserProcessList(&user_process_list); + { BlockQueryIfMemoryLimit block_query{user_process_list.user_overcommit_tracker}; user_process_list.queries.emplace(client_info.current_query_id, &res->get()); } + /// Track memory usage for all simultaneously running queries from single user. + user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user); + user_process_list.user_memory_tracker.setSoftLimit(settings.max_guaranteed_memory_usage_for_user); + user_process_list.user_memory_tracker.setDescription("(for user)"); + user_process_list.user_overcommit_tracker.setMaxWaitTime(settings.memory_usage_overcommit_max_wait_microseconds); + if (!user_process_list.user_throttler) { if (settings.max_network_bandwidth_for_user) @@ -294,10 +294,15 @@ ProcessListEntry::~ProcessListEntry() QueryStatus::QueryStatus( - ContextPtr context_, const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_) + ContextPtr context_, + const String & query_, + const ClientInfo & client_info_, + QueryPriorities::Handle && priority_handle_, + ThreadGroupStatusPtr && thread_group_) : WithContext(context_) , query(query_) , client_info(client_info_) + , thread_group(std::move(thread_group_)) , priority_handle(std::move(priority_handle_)) { auto settings = getContext()->getSettings(); diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index b1a9da314c6..258c9b2c540 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -126,7 +126,8 @@ public: ContextPtr context_, const String & query_, const ClientInfo & client_info_, - QueryPriorities::Handle && priority_handle_); + QueryPriorities::Handle && priority_handle_, + ThreadGroupStatusPtr && thread_group_); ~QueryStatus(); From 83c663e2d6d3a4b8c8790568412eaad66cab649b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 18 Jan 2022 12:21:59 +0000 Subject: [PATCH 021/164] Cleanup after code review --- programs/server/Server.cpp | 2 +- src/Common/MemoryTracker.cpp | 5 +++-- src/Common/OvercommitTracker.cpp | 23 +++++++++++++---------- src/Common/OvercommitTracker.h | 21 ++++++++++++++++++++- src/Interpreters/ProcessList.cpp | 1 - 5 files changed, 37 insertions(+), 15 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index adc6c911031..072ac942f1a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -22,7 +22,7 @@ #include #include #include -#include "Common/MemoryTracker.h" +#include #include #include #include diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index d93b13ebc27..05064e23030 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -2,7 +2,7 @@ #include #include "Common/TraceCollector.h" -#include "Common/VariableContext.h" +#include #include #include #include @@ -223,7 +223,8 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { bool need_to_throw = true; - if (!!overcommit_tracker && !!query_tracker) + bool try_to_free_momory = overcommit_tracker != nullptr && query_tracker != nullptr; + if (try_to_free_momory) need_to_throw = overcommit_tracker->needToStopQuery(query_tracker); if (need_to_throw) diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index 7ac74e761db..4be7096aa60 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -14,7 +14,7 @@ OvercommitTracker::OvercommitTracker() void OvercommitTracker::setMaxWaitTime(UInt64 wait_time) { - std::unique_lock lk(overcommit_m); + std::lock_guard guard(overcommit_m); max_wait_time = wait_time * 1us; } @@ -25,6 +25,8 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker) pickQueryToExclude(); assert(cancelation_state == QueryCancelationState::RUNNING); + // If no query was chosen we need to stop current query. + // This may happen if no soft limit is set. if (picked_tracker == nullptr) { cancelation_state = QueryCancelationState::NONE; @@ -32,7 +34,7 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker) } if (picked_tracker == tracker) return true; - return cv.wait_for(lk, max_wait_time, [this]() + return !cv.wait_for(lk, max_wait_time, [this]() { return cancelation_state == QueryCancelationState::NONE; }); @@ -43,7 +45,7 @@ void OvercommitTracker::unsubscribe(MemoryTracker * tracker) std::unique_lock lk(overcommit_m); if (picked_tracker == tracker) { - LOG_DEBUG(&Poco::Logger::get("OvercommitTracker"), "Picked query stopped"); + LOG_DEBUG(getLogger(), "Picked query stopped"); picked_tracker = nullptr; cancelation_state = QueryCancelationState::NONE; @@ -57,9 +59,10 @@ UserOvercommitTracker::UserOvercommitTracker(DB::ProcessListForUser * user_proce void UserOvercommitTracker::pickQueryToExcludeImpl() { - MemoryTracker * current_tracker = nullptr; + MemoryTracker * query_tracker = nullptr; OvercommitRatio current_ratio{0, 0}; - // At this moment query list must be read only + // At this moment query list must be read only. + // BlockQueryIfMemoryLimit is used in ProcessList to guarantee this. auto & queries = user_process_list->queries; LOG_DEBUG(logger, "Trying to choose query to stop from {} queries", queries.size()); for (auto const & query : queries) @@ -75,18 +78,18 @@ void UserOvercommitTracker::pickQueryToExcludeImpl() LOG_DEBUG(logger, "Query has ratio {}/{}", ratio.committed, ratio.soft_limit); if (ratio.soft_limit != 0 && current_ratio < ratio) { - current_tracker = memory_tracker; + query_tracker = memory_tracker; current_ratio = ratio; } } LOG_DEBUG(logger, "Selected to stop query with overcommit ratio {}/{}", current_ratio.committed, current_ratio.soft_limit); - picked_tracker = current_tracker; + picked_tracker = query_tracker; } void GlobalOvercommitTracker::pickQueryToExcludeImpl() { - MemoryTracker * current_tracker = nullptr; + MemoryTracker * query_tracker = nullptr; OvercommitRatio current_ratio{0, 0}; process_list->processEachQueryStatus([&](DB::QueryStatus const & query) { @@ -106,11 +109,11 @@ void GlobalOvercommitTracker::pickQueryToExcludeImpl() LOG_DEBUG(logger, "Query has ratio {}/{}", ratio.committed, ratio.soft_limit); if (current_ratio < ratio) { - current_tracker = memory_tracker; + query_tracker = memory_tracker; current_ratio = ratio; } }); LOG_DEBUG(logger, "Selected to stop query with overcommit ratio {}/{}", current_ratio.committed, current_ratio.soft_limit); - picked_tracker = current_tracker; + picked_tracker = query_tracker; } diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 845402c9a64..2286ad4bde2 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -3,12 +3,14 @@ #include #include #include +#include #include #include #include #include #include +// This struct is used for the comparison of query memory usage. struct OvercommitRatio { OvercommitRatio(Int64 committed_, Int64 soft_limit_) @@ -16,7 +18,7 @@ struct OvercommitRatio , soft_limit(soft_limit_) {} - friend bool operator<(OvercommitRatio const& lhs, OvercommitRatio const& rhs) noexcept + friend bool operator<(OvercommitRatio const & lhs, OvercommitRatio const & rhs) noexcept { // (a / b < c / d) <=> (a * d < c * b) return (lhs.committed * rhs.soft_limit) < (rhs.committed * lhs.soft_limit) @@ -24,12 +26,21 @@ struct OvercommitRatio || (lhs.committed == 0 && rhs.committed == 0 && lhs.soft_limit > rhs.soft_limit); } + // actual query memory usage Int64 committed; + // guaranteed amount of memory query can use Int64 soft_limit; }; class MemoryTracker; +// Usually it's hard to set some reasonable hard memory limit +// (especially, the default value). This class introduces new +// mechanisim for the limiting of memory usage. +// Soft limit represents guaranteed amount of memory query/user +// may use. It's allowed to exceed this limit. But if hard limit +// is reached, query with the biggest overcommit ratio +// is killed to free memory. struct OvercommitTracker : boost::noncopyable { OvercommitTracker(); @@ -63,6 +74,8 @@ protected: MemoryTracker * picked_tracker; QueryCancelationState cancelation_state; + virtual Poco::Logger * getLogger() = 0; + private: void pickQueryToExclude() @@ -92,6 +105,7 @@ struct UserOvercommitTracker : OvercommitTracker protected: void pickQueryToExcludeImpl() override final; + Poco::Logger * getLogger() override final { return logger; } private: DB::ProcessListForUser * user_process_list; Poco::Logger * logger = &Poco::Logger::get("UserOvercommitTracker"); @@ -108,11 +122,16 @@ struct GlobalOvercommitTracker : OvercommitTracker protected: void pickQueryToExcludeImpl() override final; + Poco::Logger * getLogger() override final { return logger; } private: DB::ProcessList * process_list; Poco::Logger * logger = &Poco::Logger::get("GlobalOvercommitTracker"); }; +// UserOvercommitTracker requires to check the whole list of user's queries +// to pick one to stop. BlockQueryIfMemoryLimit struct allows to wait until +// query selection is finished. It's used in ProcessList to make user query +// list immutable when UserOvercommitTracker reads it. struct BlockQueryIfMemoryLimit { BlockQueryIfMemoryLimit(OvercommitTracker const & overcommit_tracker) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 9d6e7b61acc..58e90e05219 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -8,7 +8,6 @@ #include #include #include -#include "Common/tests/gtest_global_context.h" #include #include #include From d7b4a32938cde0c87d34ab857af900f731a1c155 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 18 Jan 2022 20:26:12 +0000 Subject: [PATCH 022/164] fix build --- src/Interpreters/ProcessList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 24928c359e2..818b0b65eaf 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -222,7 +222,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as } auto process_it = processes.emplace(processes.end(), - query_context, query_, client_info, priorities.insert(settings.priority), std::move(thread_group)); + query_context, query_, client_info, priorities.insert(settings.priority), std::move(thread_group), query_kind); increaseQueryKindAmount(query_kind); From 3df56f46b3ff83ae05d2762506a8c41c49b2ce97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 27 Jan 2022 18:09:47 +0100 Subject: [PATCH 023/164] Fix MV view_duration_ms --- src/Processors/Transforms/buildPushingToViewsChain.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 17075e2b318..e9746f697ed 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -331,7 +331,7 @@ Chain buildPushingToViewsChain( { auto executing_inner_query = std::make_shared( storage_header, views_data->views.back(), views_data); - executing_inner_query->setRuntimeData(view_thread_status, elapsed_counter_ms); + executing_inner_query->setRuntimeData(view_thread_status, view_counter_ms); out.addSource(std::move(executing_inner_query)); } From 5f7ac9fbb12ca861da9ce6229aabe738c3fe95c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 27 Jan 2022 18:13:30 +0100 Subject: [PATCH 024/164] MV: Limit the chain threads based on parallel_view_processing --- src/Processors/Transforms/buildPushingToViewsChain.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index e9746f697ed..19302afb5c9 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -381,7 +381,7 @@ Chain buildPushingToViewsChain( processors.emplace_front(std::move(copying_data)); processors.emplace_back(std::move(finalizing_views)); result_chain = Chain(std::move(processors)); - result_chain.setNumThreads(max_parallel_streams); + result_chain.setNumThreads(std::min(views_data->max_threads, max_parallel_streams)); } if (auto * live_view = dynamic_cast(storage.get())) From c14bfc8d1c5e3ac6e790039b8379e4a00e21f469 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 27 Jan 2022 18:15:14 +0100 Subject: [PATCH 025/164] Test both fixes --- .../02187_insert_values_with_mv.reference | 12 ++++ .../02187_insert_values_with_mv.sql | 59 +++++++++++++++++++ 2 files changed, 71 insertions(+) create mode 100644 tests/queries/0_stateless/02187_insert_values_with_mv.reference create mode 100644 tests/queries/0_stateless/02187_insert_values_with_mv.sql diff --git a/tests/queries/0_stateless/02187_insert_values_with_mv.reference b/tests/queries/0_stateless/02187_insert_values_with_mv.reference new file mode 100644 index 00000000000..d24ee4faff7 --- /dev/null +++ b/tests/queries/0_stateless/02187_insert_values_with_mv.reference @@ -0,0 +1,12 @@ +VALUES 1 +TABLE 1 +VALUES 1 +VALUES 1 +VALUES 1 +VALUES 1 +VALUES 1 +TABLE 1 +TABLE 1 +TABLE 1 +TABLE 1 +TABLE 1 diff --git a/tests/queries/0_stateless/02187_insert_values_with_mv.sql b/tests/queries/0_stateless/02187_insert_values_with_mv.sql new file mode 100644 index 00000000000..91241f7f5c3 --- /dev/null +++ b/tests/queries/0_stateless/02187_insert_values_with_mv.sql @@ -0,0 +1,59 @@ +CREATE TABLE IF NOT EXISTS a (a Int64) ENGINE=Memory; +CREATE TABLE IF NOT EXISTS b (a Int64) ENGINE=Memory; +CREATE MATERIALIZED VIEW IF NOT EXISTS mv1 TO b AS Select sleepEachRow(0.05) as a FROM a; +CREATE MATERIALIZED VIEW IF NOT EXISTS mv2 TO b AS Select sleepEachRow(0.05) as a FROM a; +CREATE MATERIALIZED VIEW IF NOT EXISTS mv3 TO b AS Select sleepEachRow(0.05) as a FROM a; +CREATE MATERIALIZED VIEW IF NOT EXISTS mv4 TO b AS Select sleepEachRow(0.05) as a FROM a; +CREATE MATERIALIZED VIEW IF NOT EXISTS mv5 TO b AS Select sleepEachRow(0.05) as a FROM a; + +-- INSERT USING VALUES +INSERT INTO a VALUES (1); +-- INSERT USING TABLE +INSERT INTO a SELECT * FROM system.one; +SYSTEM FLUSH LOGS; + +SELECT 'VALUES', query_duration_ms >= 250 +FROM system.query_log +WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND query LIKE '-- INSERT USING VALUES%' + AND type = 'QueryFinish' +LIMIT 1; + +SELECT 'TABLE', query_duration_ms >= 250 +FROM system.query_log +WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND query LIKE '-- INSERT USING VALUES%' + AND type = 'QueryFinish' +LIMIT 1; + +WITH + ( + SELECT initial_query_id + FROM system.query_log + WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND query LIKE '-- INSERT USING VALUES%' + LIMIT 1 + ) AS q_id +SELECT 'VALUES', view_duration_ms >= 50 +FROM system.query_views_log +WHERE initial_query_id = q_id; + +WITH +( + SELECT initial_query_id + FROM system.query_log + WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND query LIKE '-- INSERT USING TABLE%' + LIMIT 1 +) AS q_id +SELECT 'TABLE', view_duration_ms >= 50 +FROM system.query_views_log +WHERE initial_query_id = q_id; From 676fc5b20fd5cb459a9a300f0bcc72fb902c5801 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 31 Jan 2022 17:10:59 +0000 Subject: [PATCH 026/164] Initial commit --- src/Core/Settings.h | 1 + src/Core/SettingsEnums.cpp | 20 ++++++ src/Core/SettingsEnums.h | 23 ++++++ src/Interpreters/InterpreterCreateQuery.cpp | 79 ++++++++++++++++++++- src/Parsers/ParserCreateQuery.cpp | 55 ++++++++++---- src/Parsers/ParserCreateQuery.h | 2 +- src/Storages/StorageFactory.cpp | 6 +- src/Storages/registerStorages.cpp | 6 +- 8 files changed, 172 insertions(+), 20 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d55be808aa8..f940db3085b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -421,6 +421,7 @@ class IColumn; M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic, "Default database engine.", 0) \ + M(DefaultTableEngine, default_table_engine, DefaultTableEngine::None, "Default table engine used when ENGINE is not set in CREATE statement.",0) \ M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ M(Bool, database_atomic_wait_for_drop_and_detach_synchronously, false, "When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index b62575c9730..b8ab56fcf52 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -93,6 +93,26 @@ IMPLEMENT_SETTING_ENUM_WITH_RENAME(DefaultDatabaseEngine, ErrorCodes::BAD_ARGUME {{"Ordinary", DefaultDatabaseEngine::Ordinary}, {"Atomic", DefaultDatabaseEngine::Atomic}}) +IMPLEMENT_SETTING_ENUM_WITH_RENAME(DefaultTableEngine, ErrorCodes::BAD_ARGUMENTS, + {{"Non", DefaultTableEngine::None}, + {"Log", DefaultTableEngine::Log}, + {"StripeLog", DefaultTableEngine::StripeLog}, + {"MergeTree", DefaultTableEngine::MergeTree}, + {"CollapsingMergeTree", DefaultTableEngine::CollapsingMergeTree}, + {"ReplacingMergeTree", DefaultTableEngine::ReplacingMergeTree}, + {"AggregatingMergeTree", DefaultTableEngine::AggregatingMergeTree}, + {"SummingMergeTree", DefaultTableEngine::SummingMergeTree}, + {"GraphiteMergeTree", DefaultTableEngine::GraphiteMergeTree}, + {"VersionedCollapsingMergeTree", DefaultTableEngine::VersionedCollapsingMergeTree}, + {"ReplicatedMergeTree", DefaultTableEngine::ReplicatedMergeTree}, + {"ReplicatedCollapsingMergeTree", DefaultTableEngine::ReplicatedCollapsingMergeTree}, + {"ReplicatedReplacingMergeTree", DefaultTableEngine::ReplicatedReplacingMergeTree}, + {"ReplicatedAggregatingMergeTree", DefaultTableEngine::ReplicatedAggregatingMergeTree}, + {"ReplicatedSummingMergeTree", DefaultTableEngine::ReplicatedSummingMergeTree}, + {"ReplicatedGraphiteMergeTree", DefaultTableEngine::ReplicatedGraphiteMergeTree}, + {"ReplicatedVersionedCollapsingMergeTree", DefaultTableEngine::ReplicatedVersionedCollapsingMergeTree}, + {"Memory", DefaultTableEngine::Memory}}) + IMPLEMENT_SETTING_MULTI_ENUM(MySQLDataTypesSupport, ErrorCodes::UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL, {{"decimal", MySQLDataTypesSupport::DECIMAL}, {"datetime64", MySQLDataTypesSupport::DATETIME64}}) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 106589f5d24..cbf67096768 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -120,6 +120,29 @@ enum class DefaultDatabaseEngine DECLARE_SETTING_ENUM(DefaultDatabaseEngine) +enum class DefaultTableEngine +{ + None = 0, /// Disable. Need to use ENGINE = + Log, + StripeLog, + MergeTree, + CollapsingMergeTree, + ReplacingMergeTree, + AggregatingMergeTree, + SummingMergeTree, + GraphiteMergeTree, + VersionedCollapsingMergeTree, + ReplicatedMergeTree, + ReplicatedCollapsingMergeTree, + ReplicatedReplacingMergeTree, + ReplicatedAggregatingMergeTree, + ReplicatedSummingMergeTree, + ReplicatedGraphiteMergeTree, + ReplicatedVersionedCollapsingMergeTree, + Memory, +}; + +DECLARE_SETTING_ENUM(DefaultTableEngine) enum class MySQLDataTypesSupport { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d2b77f1a439..edabb209ca1 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include "Core/SettingsEnums.h" #include #include @@ -728,11 +729,9 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (create.temporary && create.storage && create.storage->engine && create.storage->engine->name != "Memory") throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables can only be created with ENGINE = Memory, not {}", create.storage->engine->name); - - return; } - if (create.temporary) + if (create.temporary && !create.storage) { auto engine_ast = std::make_shared(); engine_ast->name = "Memory"; @@ -740,6 +739,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const auto storage_ast = std::make_shared(); storage_ast->set(storage_ast->engine, engine_ast); create.set(create.storage, storage_ast); + return; } else if (!create.as_table.empty()) { @@ -779,6 +779,79 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const create.as_table_function = as_create.as_table_function->clone(); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot set engine, it's a bug."); + return; + } + + if (!create.storage) + create.set(create.storage, std::make_shared()); + + if (getContext()->getSettingsRef().default_table_engine.value != DefaultTableEngine::None) + { + auto default_table_engine = getContext()->getSettingsRef().default_table_engine.value; + String default_table_engine_name; + switch (default_table_engine) + { + case DefaultTableEngine::Log: + default_table_engine_name = "Log"; + break; + case DefaultTableEngine::StripeLog: + default_table_engine_name = "StripeLog"; + break; + case DefaultTableEngine::MergeTree: + default_table_engine_name = "MergeTree"; + break; + case DefaultTableEngine::CollapsingMergeTree: + default_table_engine_name = "CollapsingMergeTree"; + break; + case DefaultTableEngine::ReplacingMergeTree: + default_table_engine_name = "ReplacingMergeTree"; + break; + case DefaultTableEngine::AggregatingMergeTree: + default_table_engine_name = "AggregatingMergeTree"; + break; + case DefaultTableEngine::SummingMergeTree: + default_table_engine_name = "SummingMergeTree"; + break; + case DefaultTableEngine::GraphiteMergeTree: + default_table_engine_name = "GraphiteMergeTree"; + break; + case DefaultTableEngine::VersionedCollapsingMergeTree: + default_table_engine_name = "VersionedCollapsingMergeTree"; + break; + case DefaultTableEngine::ReplicatedMergeTree: + default_table_engine_name = "ReplicatedMergeTree"; + break; + case DefaultTableEngine::ReplicatedCollapsingMergeTree: + default_table_engine_name = "ReplicatedCollapsingMergeTree"; + break; + case DefaultTableEngine::ReplicatedReplacingMergeTree: + default_table_engine_name = "ReplicatedReplacingMergeTree"; + break; + case DefaultTableEngine::ReplicatedAggregatingMergeTree: + default_table_engine_name = "ReplicatedAggregatingMergeTree"; + break; + case DefaultTableEngine::ReplicatedSummingMergeTree: + default_table_engine_name = "ReplicatedSummingMergeTree"; + break; + case DefaultTableEngine::ReplicatedGraphiteMergeTree: + default_table_engine_name = "ReplicatedGraphiteMergeTree"; + break; + case DefaultTableEngine::ReplicatedVersionedCollapsingMergeTree: + default_table_engine_name = "ReplicatedVersionedCollapsingMergeTree"; + break; + case DefaultTableEngine::Memory: + default_table_engine_name = "Memory"; + break; + default: + throw Exception( "default_table_engine is set to unknown value", ErrorCodes::BAD_ARGUMENTS); + } + if (!create.storage->engine) + { + auto engine_ast = std::make_shared(); + engine_ast->name = default_table_engine_name; + engine_ast->no_empty_args = true; + create.storage->set(create.storage->engine, engine_ast); + } } } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 6d295a0d516..fa09c9ba33f 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -18,6 +18,7 @@ #include #include #include +#include "Parsers/ASTColumnDeclaration.h" namespace DB @@ -353,20 +354,26 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr ttl_table; ASTPtr settings; - if (!s_engine.ignore(pos, expected)) - return false; + bool storage_like = false; - s_eq.ignore(pos, expected); + if (s_engine.ignore(pos, expected)) + { + s_eq.ignore(pos, expected); - if (!ident_with_optional_params_p.parse(pos, engine, expected)) - return false; + if (!ident_with_optional_params_p.parse(pos, engine, expected)) + return false; + storage_like = true; + } while (true) { if (!partition_by && s_partition_by.ignore(pos, expected)) { if (expression_p.parse(pos, partition_by, expected)) + { + storage_like = true; continue; + } else return false; } @@ -374,7 +381,10 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!primary_key && s_primary_key.ignore(pos, expected)) { if (expression_p.parse(pos, primary_key, expected)) + { + storage_like = true; continue; + } else return false; } @@ -382,7 +392,10 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!order_by && s_order_by.ignore(pos, expected)) { if (expression_p.parse(pos, order_by, expected)) + { + storage_like = true; continue; + } else return false; } @@ -390,7 +403,10 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!sample_by && s_sample_by.ignore(pos, expected)) { if (expression_p.parse(pos, sample_by, expected)) + { + storage_like = true; continue; + } else return false; } @@ -398,7 +414,10 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!ttl_table && s_ttl.ignore(pos, expected)) { if (parser_ttl_list.parse(pos, ttl_table, expected)) + { + storage_like = true; continue; + } else return false; } @@ -407,10 +426,14 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!settings_p.parse(pos, settings, expected)) return false; + storage_like = true; } break; } + // If any part of storage definition is found create storage node + if (!storage_like) + return false; auto storage = std::make_shared(); storage->set(storage->engine, engine); @@ -549,12 +572,24 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (!storage_parse_result && !is_temporary) { - if (!s_as.ignore(pos, expected)) - return false; - if (!table_function_p.parse(pos, as_table_function, expected)) + if (s_as.ignore(pos, expected) && !table_function_p.parse(pos, as_table_function, expected)) { return false; } + else + { + // ENGINE can be omitted if default_table_engine is set. + // Need to check in Interpreter + if (columns_list) + { + auto columns = columns_list->as(); + if (columns.primary_key) + { + auto storage_ast = std::make_shared(); + storage = storage_ast; + } + } + } } } /** Create queries without list of columns: @@ -590,10 +625,6 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe } } } - else if (!storage) - { - return false; - } } auto comment = parseComment(pos, expected); diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 615121eae58..d41984a240b 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -336,7 +336,7 @@ protected: /** - * ENGINE = name [PARTITION BY expr] [ORDER BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] [SETTINGS name = value, ...] + * [ENGINE = name] [PARTITION BY expr] [ORDER BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] [SETTINGS name = value, ...] */ class ParserStorage : public IParserBase { diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index eae46220c86..92f537b4f61 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -66,6 +66,10 @@ StoragePtr StorageFactory::get( bool has_force_restore_data_flag) const { String name, comment; + + if (!query.storage) + throw Exception("Incorrect CREATE query: storage required", ErrorCodes::INCORRECT_QUERY); + ASTStorage * storage_def = query.storage; bool has_engine_args = false; @@ -107,7 +111,7 @@ StoragePtr StorageFactory::get( } else { - if (!storage_def) + if (!storage_def->engine) throw Exception("Incorrect CREATE query: ENGINE required", ErrorCodes::ENGINE_REQUIRED); const ASTFunction & engine_def = *storage_def->engine; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index d61baf2eb63..f567bf6eefc 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -132,11 +132,11 @@ void registerStorages() registerStorageKafka(factory); #endif -#if USE_FILELOG + #if USE_FILELOG registerStorageFileLog(factory); -#endif + #endif -#if USE_AMQPCPP + #if USE_AMQPCPP registerStorageRabbitMQ(factory); #endif From 9b8a2ef675bfb9d26f533c44ab89beab7f148f3e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 31 Jan 2022 17:11:15 +0000 Subject: [PATCH 027/164] add initial test --- .../02184_default_table_engine.reference | 4 ++++ .../0_stateless/02184_default_table_engine.sql | 17 +++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02184_default_table_engine.reference create mode 100644 tests/queries/0_stateless/02184_default_table_engine.sql diff --git a/tests/queries/0_stateless/02184_default_table_engine.reference b/tests/queries/0_stateless/02184_default_table_engine.reference new file mode 100644 index 00000000000..642f06c7570 --- /dev/null +++ b/tests/queries/0_stateless/02184_default_table_engine.reference @@ -0,0 +1,4 @@ +x UInt8 +x UInt8 +x UInt8 +x UInt8 diff --git a/tests/queries/0_stateless/02184_default_table_engine.sql b/tests/queries/0_stateless/02184_default_table_engine.sql new file mode 100644 index 00000000000..72479d13fe0 --- /dev/null +++ b/tests/queries/0_stateless/02184_default_table_engine.sql @@ -0,0 +1,17 @@ +CREATE TABLE table_02184 (x UInt8); --{serverError 119} +SET default_table_engine = 'Log'; +CREATE TABLE table_02184 (x UInt8); +DESCRIBE TABLE table_02184; +DROP TABLE table_02184; +SET default_table_engine = 'MergeTree'; +CREATE TABLE table_02184 (x UInt8); --{serverError 42} +CREATE TABLE table_02184 (x UInt8, PRIMARY KEY (x)); +DESCRIBE TABLE table_02184; +DROP TABLE table_02184; +CREATE TABLE table_02184 (x UInt8) PARTITION BY x; --{serverError 36} +CREATE TABLE table_02184 (x UInt8) ORDER BY x; +DESCRIBE TABLE table_02184; +DROP TABLE table_02184; +CREATE TABLE table_02184 (x UInt8) PRIMARY KEY x; +DESCRIBE TABLE table_02184; +DROP TABLE table_02184; From 9c2ef98d239fff8924702626d89ac6b65585b838 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 1 Feb 2022 10:03:50 +0300 Subject: [PATCH 028/164] Apply suggestions from code review Co-authored-by: tavplubix --- src/Core/SettingsEnums.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- src/Parsers/ParserCreateQuery.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index b8ab56fcf52..d409ba0f35f 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -94,7 +94,7 @@ IMPLEMENT_SETTING_ENUM_WITH_RENAME(DefaultDatabaseEngine, ErrorCodes::BAD_ARGUME {"Atomic", DefaultDatabaseEngine::Atomic}}) IMPLEMENT_SETTING_ENUM_WITH_RENAME(DefaultTableEngine, ErrorCodes::BAD_ARGUMENTS, - {{"Non", DefaultTableEngine::None}, + {{"None", DefaultTableEngine::None}, {"Log", DefaultTableEngine::Log}, {"StripeLog", DefaultTableEngine::StripeLog}, {"MergeTree", DefaultTableEngine::MergeTree}, diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index edabb209ca1..0a7b56285ce 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -9,7 +9,7 @@ #include #include #include -#include "Core/SettingsEnums.h" +#include #include #include @@ -843,7 +843,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const default_table_engine_name = "Memory"; break; default: - throw Exception( "default_table_engine is set to unknown value", ErrorCodes::BAD_ARGUMENTS); + throw Exception( "default_table_engine is set to unknown value", ErrorCodes::LOGICAL_ERROR); } if (!create.storage->engine) { diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index fa09c9ba33f..07e5b538bb9 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -18,7 +18,7 @@ #include #include #include -#include "Parsers/ASTColumnDeclaration.h" +#include namespace DB From 1310a89fb2d01b3449e74c38ce3e54f37e1a3b73 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 1 Feb 2022 16:14:48 +0300 Subject: [PATCH 029/164] Update src/Common/MemoryTracker.cpp Co-authored-by: alesapin --- src/Common/MemoryTracker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 5f32a4d46b2..c4fc29a173d 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -190,7 +190,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { bool need_to_throw = true; - bool try_to_free_momory = overcommit_tracker != nullptr && query_tracker != nullptr; + bool try_to_free_memory = overcommit_tracker != nullptr && query_tracker != nullptr; if (try_to_free_momory) need_to_throw = overcommit_tracker->needToStopQuery(query_tracker); From 80694006f222796cabfdf05041d0eb3f660ffaec Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 1 Feb 2022 16:15:00 +0300 Subject: [PATCH 030/164] Update src/Common/MemoryTracker.cpp Co-authored-by: alesapin --- src/Common/MemoryTracker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index c4fc29a173d..4c8af23791e 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -191,7 +191,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT { bool need_to_throw = true; bool try_to_free_memory = overcommit_tracker != nullptr && query_tracker != nullptr; - if (try_to_free_momory) + if (try_to_free_memory) need_to_throw = overcommit_tracker->needToStopQuery(query_tracker); if (need_to_throw) From 759574d6f0d7190e8cdc240189e6e154967e97c3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 1 Feb 2022 16:59:41 +0000 Subject: [PATCH 031/164] improve --- src/Core/SettingsEnums.cpp | 10 ------ src/Core/SettingsEnums.h | 10 ------ src/Interpreters/InterpreterCreateQuery.cpp | 33 ++----------------- src/Parsers/ParserCreateQuery.h | 4 +-- src/Storages/StorageFactory.cpp | 6 ++-- .../02184_default_table_engine.reference | 8 ++--- .../02184_default_table_engine.sql | 8 ++--- 7 files changed, 16 insertions(+), 63 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index d409ba0f35f..bd3a4b2e46b 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -98,19 +98,9 @@ IMPLEMENT_SETTING_ENUM_WITH_RENAME(DefaultTableEngine, ErrorCodes::BAD_ARGUMENTS {"Log", DefaultTableEngine::Log}, {"StripeLog", DefaultTableEngine::StripeLog}, {"MergeTree", DefaultTableEngine::MergeTree}, - {"CollapsingMergeTree", DefaultTableEngine::CollapsingMergeTree}, {"ReplacingMergeTree", DefaultTableEngine::ReplacingMergeTree}, - {"AggregatingMergeTree", DefaultTableEngine::AggregatingMergeTree}, - {"SummingMergeTree", DefaultTableEngine::SummingMergeTree}, - {"GraphiteMergeTree", DefaultTableEngine::GraphiteMergeTree}, - {"VersionedCollapsingMergeTree", DefaultTableEngine::VersionedCollapsingMergeTree}, {"ReplicatedMergeTree", DefaultTableEngine::ReplicatedMergeTree}, - {"ReplicatedCollapsingMergeTree", DefaultTableEngine::ReplicatedCollapsingMergeTree}, {"ReplicatedReplacingMergeTree", DefaultTableEngine::ReplicatedReplacingMergeTree}, - {"ReplicatedAggregatingMergeTree", DefaultTableEngine::ReplicatedAggregatingMergeTree}, - {"ReplicatedSummingMergeTree", DefaultTableEngine::ReplicatedSummingMergeTree}, - {"ReplicatedGraphiteMergeTree", DefaultTableEngine::ReplicatedGraphiteMergeTree}, - {"ReplicatedVersionedCollapsingMergeTree", DefaultTableEngine::ReplicatedVersionedCollapsingMergeTree}, {"Memory", DefaultTableEngine::Memory}}) IMPLEMENT_SETTING_MULTI_ENUM(MySQLDataTypesSupport, ErrorCodes::UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index cbf67096768..b8a457cba80 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -126,19 +126,9 @@ enum class DefaultTableEngine Log, StripeLog, MergeTree, - CollapsingMergeTree, ReplacingMergeTree, - AggregatingMergeTree, - SummingMergeTree, - GraphiteMergeTree, - VersionedCollapsingMergeTree, ReplicatedMergeTree, - ReplicatedCollapsingMergeTree, ReplicatedReplacingMergeTree, - ReplicatedAggregatingMergeTree, - ReplicatedSummingMergeTree, - ReplicatedGraphiteMergeTree, - ReplicatedVersionedCollapsingMergeTree, Memory, }; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 0a7b56285ce..dc0d5945c72 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -782,6 +782,9 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const return; } + if (create.is_ordinary_view || create.is_live_view || create.is_window_view || create.is_dictionary) + return; + if (!create.storage) create.set(create.storage, std::make_shared()); @@ -800,45 +803,15 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const case DefaultTableEngine::MergeTree: default_table_engine_name = "MergeTree"; break; - case DefaultTableEngine::CollapsingMergeTree: - default_table_engine_name = "CollapsingMergeTree"; - break; case DefaultTableEngine::ReplacingMergeTree: default_table_engine_name = "ReplacingMergeTree"; break; - case DefaultTableEngine::AggregatingMergeTree: - default_table_engine_name = "AggregatingMergeTree"; - break; - case DefaultTableEngine::SummingMergeTree: - default_table_engine_name = "SummingMergeTree"; - break; - case DefaultTableEngine::GraphiteMergeTree: - default_table_engine_name = "GraphiteMergeTree"; - break; - case DefaultTableEngine::VersionedCollapsingMergeTree: - default_table_engine_name = "VersionedCollapsingMergeTree"; - break; case DefaultTableEngine::ReplicatedMergeTree: default_table_engine_name = "ReplicatedMergeTree"; break; - case DefaultTableEngine::ReplicatedCollapsingMergeTree: - default_table_engine_name = "ReplicatedCollapsingMergeTree"; - break; case DefaultTableEngine::ReplicatedReplacingMergeTree: default_table_engine_name = "ReplicatedReplacingMergeTree"; break; - case DefaultTableEngine::ReplicatedAggregatingMergeTree: - default_table_engine_name = "ReplicatedAggregatingMergeTree"; - break; - case DefaultTableEngine::ReplicatedSummingMergeTree: - default_table_engine_name = "ReplicatedSummingMergeTree"; - break; - case DefaultTableEngine::ReplicatedGraphiteMergeTree: - default_table_engine_name = "ReplicatedGraphiteMergeTree"; - break; - case DefaultTableEngine::ReplicatedVersionedCollapsingMergeTree: - default_table_engine_name = "ReplicatedVersionedCollapsingMergeTree"; - break; case DefaultTableEngine::Memory: default_table_engine_name = "Memory"; break; diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index d41984a240b..c48cea9c480 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -276,7 +276,7 @@ protected: class ParserIndexDeclaration : public IParserBase { public: - ParserIndexDeclaration() {} + ParserIndexDeclaration() = default; protected: const char * getName() const override { return "index declaration"; } @@ -391,7 +391,7 @@ class ParserTableOverrideDeclaration : public IParserBase { public: const bool is_standalone; - ParserTableOverrideDeclaration(bool is_standalone_ = true) : is_standalone(is_standalone_) { } + explicit ParserTableOverrideDeclaration(bool is_standalone_ = true) : is_standalone(is_standalone_) { } protected: const char * getName() const override { return "table override declaration"; } diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 92f537b4f61..c2f6fb1608d 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -67,9 +67,6 @@ StoragePtr StorageFactory::get( { String name, comment; - if (!query.storage) - throw Exception("Incorrect CREATE query: storage required", ErrorCodes::INCORRECT_QUERY); - ASTStorage * storage_def = query.storage; bool has_engine_args = false; @@ -111,6 +108,9 @@ StoragePtr StorageFactory::get( } else { + if (!query.storage) + throw Exception("Incorrect CREATE query: storage required", ErrorCodes::INCORRECT_QUERY); + if (!storage_def->engine) throw Exception("Incorrect CREATE query: ENGINE required", ErrorCodes::ENGINE_REQUIRED); diff --git a/tests/queries/0_stateless/02184_default_table_engine.reference b/tests/queries/0_stateless/02184_default_table_engine.reference index 642f06c7570..5c92ff84d53 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.reference +++ b/tests/queries/0_stateless/02184_default_table_engine.reference @@ -1,4 +1,4 @@ -x UInt8 -x UInt8 -x UInt8 -x UInt8 +CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = Log +CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = MergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = MergeTree\nORDER BY x\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = MergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02184_default_table_engine.sql b/tests/queries/0_stateless/02184_default_table_engine.sql index 72479d13fe0..e6925ac7b04 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.sql +++ b/tests/queries/0_stateless/02184_default_table_engine.sql @@ -1,17 +1,17 @@ CREATE TABLE table_02184 (x UInt8); --{serverError 119} SET default_table_engine = 'Log'; CREATE TABLE table_02184 (x UInt8); -DESCRIBE TABLE table_02184; +SHOW CREATE TABLE table_02184; DROP TABLE table_02184; SET default_table_engine = 'MergeTree'; CREATE TABLE table_02184 (x UInt8); --{serverError 42} CREATE TABLE table_02184 (x UInt8, PRIMARY KEY (x)); -DESCRIBE TABLE table_02184; +SHOW CREATE TABLE table_02184; DROP TABLE table_02184; CREATE TABLE table_02184 (x UInt8) PARTITION BY x; --{serverError 36} CREATE TABLE table_02184 (x UInt8) ORDER BY x; -DESCRIBE TABLE table_02184; +SHOW CREATE TABLE table_02184; DROP TABLE table_02184; CREATE TABLE table_02184 (x UInt8) PRIMARY KEY x; -DESCRIBE TABLE table_02184; +SHOW CREATE TABLE table_02184; DROP TABLE table_02184; From 8c822755cbd21c08b565228deab597d8c3a3de8b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 2 Feb 2022 12:33:29 +0000 Subject: [PATCH 032/164] fix AS --- src/Interpreters/InterpreterCreateQuery.cpp | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index dc0d5945c72..3d4e51ed21e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -773,12 +773,13 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const "Cannot CREATE a table AS " + qualified_name + ", it is a Dictionary", ErrorCodes::INCORRECT_QUERY); - if (as_create.storage) - create.set(create.storage, as_create.storage->ptr()); - else if (as_create.as_table_function) - create.as_table_function = as_create.as_table_function->clone(); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot set engine, it's a bug."); + if (!create.storage) + { + if (as_create.storage) + create.set(create.storage, as_create.storage->ptr()); + else if (as_create.as_table_function) + create.as_table_function = as_create.as_table_function->clone(); + } return; } From cce0452d192902a76ff2553ed73e73ea485100c0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 2 Feb 2022 18:07:03 +0000 Subject: [PATCH 033/164] add more tests, fix style, fix MATERIALIZED VIEW POPULATE without ENGINE --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Parsers/ParserCreateQuery.cpp | 4 +- .../02184_default_table_engine.reference | 1015 +++++++++++++++++ .../02184_default_table_engine.sql | 26 + 4 files changed, 1044 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3d4e51ed21e..535b9c07bc6 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -817,7 +817,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const default_table_engine_name = "Memory"; break; default: - throw Exception( "default_table_engine is set to unknown value", ErrorCodes::LOGICAL_ERROR); + throw Exception("default_table_engine is set to unknown value", ErrorCodes::LOGICAL_ERROR); } if (!create.storage->engine) { diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 07e5b538bb9..0653e1632b8 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -1294,8 +1294,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (is_materialized_view && !to_table) { /// Internal ENGINE for MATERIALIZED VIEW must be specified. - if (!storage_p.parse(pos, storage, expected)) - return false; + /// Actually check it in Interpreter as default_table_engine can be set + storage_p.parse(pos, storage, expected); if (s_populate.ignore(pos, expected)) is_populate = true; diff --git a/tests/queries/0_stateless/02184_default_table_engine.reference b/tests/queries/0_stateless/02184_default_table_engine.reference index 5c92ff84d53..4e5264af36b 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.reference +++ b/tests/queries/0_stateless/02184_default_table_engine.reference @@ -1,4 +1,1019 @@ CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = Log CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = MergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +CREATE TABLE default.test_optimize_exception\n(\n `date` Date\n)\nENGINE = MergeTree\nPARTITION BY toYYYYMM(date)\nORDER BY date\nSETTINGS index_granularity = 8192 CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = MergeTree\nORDER BY x\nSETTINGS index_granularity = 8192 CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = MergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +CREATE TABLE default.numbers1\n(\n `number` UInt64\n)\nENGINE = Memory +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +30 +31 +32 +33 +34 +35 +36 +37 +38 +39 +40 +41 +42 +43 +44 +45 +46 +47 +48 +49 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 +60 +61 +62 +63 +64 +65 +66 +67 +68 +69 +70 +71 +72 +73 +74 +75 +76 +77 +78 +79 +80 +81 +82 +83 +84 +85 +86 +87 +88 +89 +90 +91 +92 +93 +94 +95 +96 +97 +98 +99 +100 +101 +102 +103 +104 +105 +106 +107 +108 +109 +110 +111 +112 +113 +114 +115 +116 +117 +118 +119 +120 +121 +122 +123 +124 +125 +126 +127 +128 +129 +130 +131 +132 +133 +134 +135 +136 +137 +138 +139 +140 +141 +142 +143 +144 +145 +146 +147 +148 +149 +150 +151 +152 +153 +154 +155 +156 +157 +158 +159 +160 +161 +162 +163 +164 +165 +166 +167 +168 +169 +170 +171 +172 +173 +174 +175 +176 +177 +178 +179 +180 +181 +182 +183 +184 +185 +186 +187 +188 +189 +190 +191 +192 +193 +194 +195 +196 +197 +198 +199 +200 +201 +202 +203 +204 +205 +206 +207 +208 +209 +210 +211 +212 +213 +214 +215 +216 +217 +218 +219 +220 +221 +222 +223 +224 +225 +226 +227 +228 +229 +230 +231 +232 +233 +234 +235 +236 +237 +238 +239 +240 +241 +242 +243 +244 +245 +246 +247 +248 +249 +250 +251 +252 +253 +254 +255 +256 +257 +258 +259 +260 +261 +262 +263 +264 +265 +266 +267 +268 +269 +270 +271 +272 +273 +274 +275 +276 +277 +278 +279 +280 +281 +282 +283 +284 +285 +286 +287 +288 +289 +290 +291 +292 +293 +294 +295 +296 +297 +298 +299 +300 +301 +302 +303 +304 +305 +306 +307 +308 +309 +310 +311 +312 +313 +314 +315 +316 +317 +318 +319 +320 +321 +322 +323 +324 +325 +326 +327 +328 +329 +330 +331 +332 +333 +334 +335 +336 +337 +338 +339 +340 +341 +342 +343 +344 +345 +346 +347 +348 +349 +350 +351 +352 +353 +354 +355 +356 +357 +358 +359 +360 +361 +362 +363 +364 +365 +366 +367 +368 +369 +370 +371 +372 +373 +374 +375 +376 +377 +378 +379 +380 +381 +382 +383 +384 +385 +386 +387 +388 +389 +390 +391 +392 +393 +394 +395 +396 +397 +398 +399 +400 +401 +402 +403 +404 +405 +406 +407 +408 +409 +410 +411 +412 +413 +414 +415 +416 +417 +418 +419 +420 +421 +422 +423 +424 +425 +426 +427 +428 +429 +430 +431 +432 +433 +434 +435 +436 +437 +438 +439 +440 +441 +442 +443 +444 +445 +446 +447 +448 +449 +450 +451 +452 +453 +454 +455 +456 +457 +458 +459 +460 +461 +462 +463 +464 +465 +466 +467 +468 +469 +470 +471 +472 +473 +474 +475 +476 +477 +478 +479 +480 +481 +482 +483 +484 +485 +486 +487 +488 +489 +490 +491 +492 +493 +494 +495 +496 +497 +498 +499 +500 +501 +502 +503 +504 +505 +506 +507 +508 +509 +510 +511 +512 +513 +514 +515 +516 +517 +518 +519 +520 +521 +522 +523 +524 +525 +526 +527 +528 +529 +530 +531 +532 +533 +534 +535 +536 +537 +538 +539 +540 +541 +542 +543 +544 +545 +546 +547 +548 +549 +550 +551 +552 +553 +554 +555 +556 +557 +558 +559 +560 +561 +562 +563 +564 +565 +566 +567 +568 +569 +570 +571 +572 +573 +574 +575 +576 +577 +578 +579 +580 +581 +582 +583 +584 +585 +586 +587 +588 +589 +590 +591 +592 +593 +594 +595 +596 +597 +598 +599 +600 +601 +602 +603 +604 +605 +606 +607 +608 +609 +610 +611 +612 +613 +614 +615 +616 +617 +618 +619 +620 +621 +622 +623 +624 +625 +626 +627 +628 +629 +630 +631 +632 +633 +634 +635 +636 +637 +638 +639 +640 +641 +642 +643 +644 +645 +646 +647 +648 +649 +650 +651 +652 +653 +654 +655 +656 +657 +658 +659 +660 +661 +662 +663 +664 +665 +666 +667 +668 +669 +670 +671 +672 +673 +674 +675 +676 +677 +678 +679 +680 +681 +682 +683 +684 +685 +686 +687 +688 +689 +690 +691 +692 +693 +694 +695 +696 +697 +698 +699 +700 +701 +702 +703 +704 +705 +706 +707 +708 +709 +710 +711 +712 +713 +714 +715 +716 +717 +718 +719 +720 +721 +722 +723 +724 +725 +726 +727 +728 +729 +730 +731 +732 +733 +734 +735 +736 +737 +738 +739 +740 +741 +742 +743 +744 +745 +746 +747 +748 +749 +750 +751 +752 +753 +754 +755 +756 +757 +758 +759 +760 +761 +762 +763 +764 +765 +766 +767 +768 +769 +770 +771 +772 +773 +774 +775 +776 +777 +778 +779 +780 +781 +782 +783 +784 +785 +786 +787 +788 +789 +790 +791 +792 +793 +794 +795 +796 +797 +798 +799 +800 +801 +802 +803 +804 +805 +806 +807 +808 +809 +810 +811 +812 +813 +814 +815 +816 +817 +818 +819 +820 +821 +822 +823 +824 +825 +826 +827 +828 +829 +830 +831 +832 +833 +834 +835 +836 +837 +838 +839 +840 +841 +842 +843 +844 +845 +846 +847 +848 +849 +850 +851 +852 +853 +854 +855 +856 +857 +858 +859 +860 +861 +862 +863 +864 +865 +866 +867 +868 +869 +870 +871 +872 +873 +874 +875 +876 +877 +878 +879 +880 +881 +882 +883 +884 +885 +886 +887 +888 +889 +890 +891 +892 +893 +894 +895 +896 +897 +898 +899 +900 +901 +902 +903 +904 +905 +906 +907 +908 +909 +910 +911 +912 +913 +914 +915 +916 +917 +918 +919 +920 +921 +922 +923 +924 +925 +926 +927 +928 +929 +930 +931 +932 +933 +934 +935 +936 +937 +938 +939 +940 +941 +942 +943 +944 +945 +946 +947 +948 +949 +950 +951 +952 +953 +954 +955 +956 +957 +958 +959 +960 +961 +962 +963 +964 +965 +966 +967 +968 +969 +970 +971 +972 +973 +974 +975 +976 +977 +978 +979 +980 +981 +982 +983 +984 +985 +986 +987 +988 +989 +990 +991 +992 +993 +994 +995 +996 +997 +998 +999 +9 +6 +3 +1 +8 +2 +5 +7 +4 +0 +CREATE TABLE default.numbers2\n(\n `number` UInt64\n)\nENGINE = MergeTree\nORDER BY intHash32(number)\nSAMPLE BY intHash32(number)\nSETTINGS index_granularity = 8192 +2014-01-02 0 0 1970-01-01 03:00:00 2014-01-02 03:04:06 +1 2014-01-02 07:04:06 diff --git a/tests/queries/0_stateless/02184_default_table_engine.sql b/tests/queries/0_stateless/02184_default_table_engine.sql index e6925ac7b04..fe52fbcec0b 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.sql +++ b/tests/queries/0_stateless/02184_default_table_engine.sql @@ -8,6 +8,9 @@ CREATE TABLE table_02184 (x UInt8); --{serverError 42} CREATE TABLE table_02184 (x UInt8, PRIMARY KEY (x)); SHOW CREATE TABLE table_02184; DROP TABLE table_02184; +CREATE TABLE test_optimize_exception (date Date) PARTITION BY toYYYYMM(date) ORDER BY date; +SHOW CREATE TABLE test_optimize_exception; +DROP TABLE test_optimize_exception; CREATE TABLE table_02184 (x UInt8) PARTITION BY x; --{serverError 36} CREATE TABLE table_02184 (x UInt8) ORDER BY x; SHOW CREATE TABLE table_02184; @@ -15,3 +18,26 @@ DROP TABLE table_02184; CREATE TABLE table_02184 (x UInt8) PRIMARY KEY x; SHOW CREATE TABLE table_02184; DROP TABLE table_02184; +SET default_table_engine = 'Memory'; +CREATE TABLE numbers1 AS SELECT number FROM numbers(1000); +SHOW CREATE TABLE numbers1; +SELECT * FROM numbers1; +DROP TABLE numbers1; +SET default_table_engine = 'MergeTree'; +CREATE TABLE numbers2 ORDER BY intHash32(number) SAMPLE BY intHash32(number) AS SELECT number FROM numbers(10); +SELECT * FROM numbers2; +SHOW CREATE TABLE numbers2; +DROP TABLE numbers2; +CREATE TABLE test_table (EventDate Date, CounterID UInt32, UserID UInt64, EventTime DateTime('Europe/Moscow'), UTCEventTime DateTime('UTC')) PARTITION BY EventDate PRIMARY KEY CounterID; +SET default_table_engine = 'Memory'; +CREATE MATERIALIZED VIEW test_view (Rows UInt64, MaxHitTime DateTime('Europe/Moscow')) AS SELECT count() AS Rows, max(UTCEventTime) AS MaxHitTime FROM test_table; +CREATE MATERIALIZED VIEW test_view_filtered (EventDate Date, CounterID UInt32) POPULATE AS SELECT CounterID, EventDate FROM test_table WHERE EventDate < '2013-01-01'; +INSERT INTO test_table (EventDate, UTCEventTime) VALUES ('2014-01-02', '2014-01-02 03:04:06'); + +SELECT * FROM test_table; +SELECT * FROM test_view; +SELECT * FROM test_view_filtered; + +DROP TABLE test_table; +DROP TABLE test_view; +DROP TABLE test_view_filtered; \ No newline at end of file From e74fec774513230e9196e4d0ac548668d1b75241 Mon Sep 17 00:00:00 2001 From: Gaurav Kumar Date: Fri, 4 Feb 2022 09:53:03 -0800 Subject: [PATCH 034/164] Provide important performance hint max_rows_in_set this is important when global in queries return large amount of data --- docs/en/sql-reference/operators/in.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 3d8d2673468..5ba9b3e1f39 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -216,6 +216,19 @@ This is more optimal than using the normal IN. However, keep the following point It also makes sense to specify a local table in the `GLOBAL IN` clause, in case this local table is only available on the requestor server and you want to use data from it on remote servers. +### Distributed Subqueries and max_rows_in_set + +You can use [`max_rows_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) and [`max_bytes_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) to control how much data is tranferred during distributed queries. + +This is specially important if the `global in` query returns a large amount of data. Consider the following sql - +```sql + +select * from table1 where col1 global in (select col1 from table2 where ) + +``` +If `some_predicate` is not selective enough, it will return large amount of data and cause performance issues. In such cases, it is wise to limit the data transfer over the network. Also, note that `set_overflow_mode` is set to `throw` meaning that an expception is raised when these thresolds are met + + ### Distributed Subqueries and max_parallel_replicas {#max_parallel_replica-subqueries} When max_parallel_replicas is greater than 1, distributed queries are further transformed. For example, the following: From fc800bf19106091f84d6821ff5addfc0beda290c Mon Sep 17 00:00:00 2001 From: Gaurav Kumar Date: Fri, 4 Feb 2022 10:02:05 -0800 Subject: [PATCH 035/164] reference to distributed queries processing The reading section is missing important link to how distributed queries `in` queries are processed --- docs/en/engines/table-engines/special/distributed.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index faa1026b919..36aeaf7c667 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -209,6 +209,8 @@ When querying a `Distributed` table, `SELECT` queries are sent to all shards and When the `max_parallel_replicas` option is enabled, query processing is parallelized across all replicas within a single shard. For more information, see the section [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). +To learn more about how distibuted `in` and `global in` queries are processed, refer to [this](../../../sql-reference/operators/in.md/#select-distributed-subqueries) documentation. + ## Virtual Columns {#virtual-columns} - `_shard_num` — Contains the `shard_num` value from the table `system.clusters`. Type: [UInt32](../../../sql-reference/data-types/int-uint.md). From ba53bbbeefc8e44724e130a55c8ef3985554d2b8 Mon Sep 17 00:00:00 2001 From: Gaurav Kumar Date: Fri, 4 Feb 2022 10:24:45 -0800 Subject: [PATCH 036/164] Update in.md --- docs/en/sql-reference/operators/in.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 5ba9b3e1f39..3632ab70200 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -226,7 +226,7 @@ This is specially important if the `global in` query returns a large amount of select * from table1 where col1 global in (select col1 from table2 where ) ``` -If `some_predicate` is not selective enough, it will return large amount of data and cause performance issues. In such cases, it is wise to limit the data transfer over the network. Also, note that `set_overflow_mode` is set to `throw` meaning that an expception is raised when these thresolds are met +If `some_predicate` is not selective enough, it will return large amount of data and cause performance issues. In such cases, it is wise to limit the data transfer over the network. Also, note that `set_overflow_mode` is set to `throw` (by default) meaning that an expception is raised when these thresolds are met ### Distributed Subqueries and max_parallel_replicas {#max_parallel_replica-subqueries} From 0c47dd4ec2eacfbc0c559e6a21ef156e102a0522 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 6 Feb 2022 12:17:03 +0800 Subject: [PATCH 037/164] format override input-format --- programs/local/LocalServer.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index a6c9a6a4524..2217f2bbdbe 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -337,8 +337,9 @@ std::string LocalServer::getInitialCreateTableQuery() format_from_file_name = FormatFactory::instance().getFormatFromFileName(file_name, false); } - auto data_format - = backQuoteIfNeed(config().getString("table-data-format", format_from_file_name.empty() ? "TSV" : format_from_file_name)); + auto data_format = backQuoteIfNeed( + config().getString("table-data-format", config().getString("format", format_from_file_name.empty() ? "TSV" : format_from_file_name))); + if (table_structure == "auto") table_structure = ""; From d5a2f725c27a9a5ee2992e711eeea547c451b357 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 7 Feb 2022 12:38:54 +0000 Subject: [PATCH 038/164] better tests --- .../02184_default_table_engine.reference | 1012 +---------------- .../02184_default_table_engine.sql | 10 +- 2 files changed, 10 insertions(+), 1012 deletions(-) diff --git a/tests/queries/0_stateless/02184_default_table_engine.reference b/tests/queries/0_stateless/02184_default_table_engine.reference index 4e5264af36b..daa97739757 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.reference +++ b/tests/queries/0_stateless/02184_default_table_engine.reference @@ -4,1016 +4,10 @@ CREATE TABLE default.test_optimize_exception\n(\n `date` Date\n)\nENGINE = Me CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = MergeTree\nORDER BY x\nSETTINGS index_granularity = 8192 CREATE TABLE default.table_02184\n(\n `x` UInt8\n)\nENGINE = MergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 CREATE TABLE default.numbers1\n(\n `number` UInt64\n)\nENGINE = Memory -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 -10 -11 -12 -13 -14 -15 -16 -17 -18 -19 -20 -21 -22 -23 -24 -25 -26 -27 -28 -29 -30 -31 -32 -33 -34 -35 -36 -37 -38 -39 -40 -41 -42 -43 -44 +4.5 45 -46 -47 -48 -49 -50 -51 -52 -53 -54 -55 -56 -57 -58 -59 -60 -61 -62 -63 -64 -65 -66 -67 -68 -69 -70 -71 -72 -73 -74 -75 -76 -77 -78 -79 -80 -81 -82 -83 -84 -85 -86 -87 -88 -89 -90 -91 -92 -93 -94 -95 -96 -97 -98 -99 -100 -101 -102 -103 -104 -105 -106 -107 -108 -109 -110 -111 -112 -113 -114 -115 -116 -117 -118 -119 -120 -121 -122 -123 -124 -125 -126 -127 -128 -129 -130 -131 -132 -133 -134 -135 -136 -137 -138 -139 -140 -141 -142 -143 -144 -145 -146 -147 -148 -149 -150 -151 -152 -153 -154 -155 -156 -157 -158 -159 -160 -161 -162 -163 -164 -165 -166 -167 -168 -169 -170 -171 -172 -173 -174 -175 -176 -177 -178 -179 -180 -181 -182 -183 -184 -185 -186 -187 -188 -189 -190 -191 -192 -193 -194 -195 -196 -197 -198 -199 -200 -201 -202 -203 -204 -205 -206 -207 -208 -209 -210 -211 -212 -213 -214 -215 -216 -217 -218 -219 -220 -221 -222 -223 -224 -225 -226 -227 -228 -229 -230 -231 -232 -233 -234 -235 -236 -237 -238 -239 -240 -241 -242 -243 -244 -245 -246 -247 -248 -249 -250 -251 -252 -253 -254 -255 -256 -257 -258 -259 -260 -261 -262 -263 -264 -265 -266 -267 -268 -269 -270 -271 -272 -273 -274 -275 -276 -277 -278 -279 -280 -281 -282 -283 -284 -285 -286 -287 -288 -289 -290 -291 -292 -293 -294 -295 -296 -297 -298 -299 -300 -301 -302 -303 -304 -305 -306 -307 -308 -309 -310 -311 -312 -313 -314 -315 -316 -317 -318 -319 -320 -321 -322 -323 -324 -325 -326 -327 -328 -329 -330 -331 -332 -333 -334 -335 -336 -337 -338 -339 -340 -341 -342 -343 -344 -345 -346 -347 -348 -349 -350 -351 -352 -353 -354 -355 -356 -357 -358 -359 -360 -361 -362 -363 -364 -365 -366 -367 -368 -369 -370 -371 -372 -373 -374 -375 -376 -377 -378 -379 -380 -381 -382 -383 -384 -385 -386 -387 -388 -389 -390 -391 -392 -393 -394 -395 -396 -397 -398 -399 -400 -401 -402 -403 -404 -405 -406 -407 -408 -409 -410 -411 -412 -413 -414 -415 -416 -417 -418 -419 -420 -421 -422 -423 -424 -425 -426 -427 -428 -429 -430 -431 -432 -433 -434 -435 -436 -437 -438 -439 -440 -441 -442 -443 -444 -445 -446 -447 -448 -449 -450 -451 -452 -453 -454 -455 -456 -457 -458 -459 -460 -461 -462 -463 -464 -465 -466 -467 -468 -469 -470 -471 -472 -473 -474 -475 -476 -477 -478 -479 -480 -481 -482 -483 -484 -485 -486 -487 -488 -489 -490 -491 -492 -493 -494 -495 -496 -497 -498 -499 -500 -501 -502 -503 -504 -505 -506 -507 -508 -509 -510 -511 -512 -513 -514 -515 -516 -517 -518 -519 -520 -521 -522 -523 -524 -525 -526 -527 -528 -529 -530 -531 -532 -533 -534 -535 -536 -537 -538 -539 -540 -541 -542 -543 -544 -545 -546 -547 -548 -549 -550 -551 -552 -553 -554 -555 -556 -557 -558 -559 -560 -561 -562 -563 -564 -565 -566 -567 -568 -569 -570 -571 -572 -573 -574 -575 -576 -577 -578 -579 -580 -581 -582 -583 -584 -585 -586 -587 -588 -589 -590 -591 -592 -593 -594 -595 -596 -597 -598 -599 -600 -601 -602 -603 -604 -605 -606 -607 -608 -609 -610 -611 -612 -613 -614 -615 -616 -617 -618 -619 -620 -621 -622 -623 -624 -625 -626 -627 -628 -629 -630 -631 -632 -633 -634 -635 -636 -637 -638 -639 -640 -641 -642 -643 -644 -645 -646 -647 -648 -649 -650 -651 -652 -653 -654 -655 -656 -657 -658 -659 -660 -661 -662 -663 -664 -665 -666 -667 -668 -669 -670 -671 -672 -673 -674 -675 -676 -677 -678 -679 -680 -681 -682 -683 -684 -685 -686 -687 -688 -689 -690 -691 -692 -693 -694 -695 -696 -697 -698 -699 -700 -701 -702 -703 -704 -705 -706 -707 -708 -709 -710 -711 -712 -713 -714 -715 -716 -717 -718 -719 -720 -721 -722 -723 -724 -725 -726 -727 -728 -729 -730 -731 -732 -733 -734 -735 -736 -737 -738 -739 -740 -741 -742 -743 -744 -745 -746 -747 -748 -749 -750 -751 -752 -753 -754 -755 -756 -757 -758 -759 -760 -761 -762 -763 -764 -765 -766 -767 -768 -769 -770 -771 -772 -773 -774 -775 -776 -777 -778 -779 -780 -781 -782 -783 -784 -785 -786 -787 -788 -789 -790 -791 -792 -793 -794 -795 -796 -797 -798 -799 -800 -801 -802 -803 -804 -805 -806 -807 -808 -809 -810 -811 -812 -813 -814 -815 -816 -817 -818 -819 -820 -821 -822 -823 -824 -825 -826 -827 -828 -829 -830 -831 -832 -833 -834 -835 -836 -837 -838 -839 -840 -841 -842 -843 -844 -845 -846 -847 -848 -849 -850 -851 -852 -853 -854 -855 -856 -857 -858 -859 -860 -861 -862 -863 -864 -865 -866 -867 -868 -869 -870 -871 -872 -873 -874 -875 -876 -877 -878 -879 -880 -881 -882 -883 -884 -885 -886 -887 -888 -889 -890 -891 -892 -893 -894 -895 -896 -897 -898 -899 -900 -901 -902 -903 -904 -905 -906 -907 -908 -909 -910 -911 -912 -913 -914 -915 -916 -917 -918 -919 -920 -921 -922 -923 -924 -925 -926 -927 -928 -929 -930 -931 -932 -933 -934 -935 -936 -937 -938 -939 -940 -941 -942 -943 -944 -945 -946 -947 -948 -949 -950 -951 -952 -953 -954 -955 -956 -957 -958 -959 -960 -961 -962 -963 -964 -965 -966 -967 -968 -969 -970 -971 -972 -973 -974 -975 -976 -977 -978 -979 -980 -981 -982 -983 -984 -985 -986 -987 -988 -989 -990 -991 -992 -993 -994 -995 -996 -997 -998 -999 -9 -6 -3 -1 -8 -2 -5 -7 -4 -0 CREATE TABLE default.numbers2\n(\n `number` UInt64\n)\nENGINE = MergeTree\nORDER BY intHash32(number)\nSAMPLE BY intHash32(number)\nSETTINGS index_granularity = 8192 +45 +CREATE TABLE default.numbers3\n(\n `number` UInt64\n)\nENGINE = Log 2014-01-02 0 0 1970-01-01 03:00:00 2014-01-02 03:04:06 1 2014-01-02 07:04:06 diff --git a/tests/queries/0_stateless/02184_default_table_engine.sql b/tests/queries/0_stateless/02184_default_table_engine.sql index fe52fbcec0b..496e79de23a 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.sql +++ b/tests/queries/0_stateless/02184_default_table_engine.sql @@ -19,15 +19,19 @@ CREATE TABLE table_02184 (x UInt8) PRIMARY KEY x; SHOW CREATE TABLE table_02184; DROP TABLE table_02184; SET default_table_engine = 'Memory'; -CREATE TABLE numbers1 AS SELECT number FROM numbers(1000); +CREATE TABLE numbers1 AS SELECT number FROM numbers(10); SHOW CREATE TABLE numbers1; -SELECT * FROM numbers1; +SELECT avg(number) FROM numbers1; DROP TABLE numbers1; SET default_table_engine = 'MergeTree'; CREATE TABLE numbers2 ORDER BY intHash32(number) SAMPLE BY intHash32(number) AS SELECT number FROM numbers(10); -SELECT * FROM numbers2; +SELECT sum(number) FROM numbers2; SHOW CREATE TABLE numbers2; DROP TABLE numbers2; +CREATE TABLE numbers3 ENGINE = Log AS SELECT number FROM numbers(10); +SELECT sum(number) FROM numbers3; +SHOW CREATE TABLE numbers3; +DROP TABLE numbers3; CREATE TABLE test_table (EventDate Date, CounterID UInt32, UserID UInt64, EventTime DateTime('Europe/Moscow'), UTCEventTime DateTime('UTC')) PARTITION BY EventDate PRIMARY KEY CounterID; SET default_table_engine = 'Memory'; CREATE MATERIALIZED VIEW test_view (Rows UInt64, MaxHitTime DateTime('Europe/Moscow')) AS SELECT count() AS Rows, max(UTCEventTime) AS MaxHitTime FROM test_table; From 7e54dafdc15c84d49048bcbd715e3b381078eaca Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 8 Feb 2022 10:41:07 +0000 Subject: [PATCH 039/164] Fix wrong destruction order in CreatingSetsTransform. --- src/Processors/Transforms/CreatingSetsTransform.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index 839ab0cac88..48a32ea8663 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -44,8 +44,8 @@ public: private: SubqueryForSet subquery; - std::unique_ptr executor; QueryPipeline table_out; + std::unique_ptr executor; UInt64 read_rows = 0; Stopwatch watch; From b680096f876e167cd9e91f2213be9bc5a7083281 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 8 Feb 2022 12:09:01 +0000 Subject: [PATCH 040/164] Add test for global overcommit tracker --- .../__init__.py | 0 .../configs/config.xml | 3 ++ .../test_global_overcommit_tracker/test.py | 43 +++++++++++++++++++ 3 files changed, 46 insertions(+) create mode 100644 tests/integration/test_global_overcommit_tracker/__init__.py create mode 100644 tests/integration/test_global_overcommit_tracker/configs/config.xml create mode 100644 tests/integration/test_global_overcommit_tracker/test.py diff --git a/tests/integration/test_global_overcommit_tracker/__init__.py b/tests/integration/test_global_overcommit_tracker/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_global_overcommit_tracker/configs/config.xml b/tests/integration/test_global_overcommit_tracker/configs/config.xml new file mode 100644 index 00000000000..10c21b03f83 --- /dev/null +++ b/tests/integration/test_global_overcommit_tracker/configs/config.xml @@ -0,0 +1,3 @@ + + 5000000 + \ No newline at end of file diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py new file mode 100644 index 00000000000..e419cb8ad64 --- /dev/null +++ b/tests/integration/test_global_overcommit_tracker/test.py @@ -0,0 +1,43 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance('node', main_configs=['configs/config.xml']) + +@pytest.fixture(scope='module', autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +TEST_QUERY_A = 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user=1,memory_usage_overcommit_max_wait_microseconds=500' +TEST_QUERY_B = 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user=2,memory_usage_overcommit_max_wait_microseconds=500' + +def test_overcommited_is_killed(): + node.query("CREATE USER A") + node.query("GRANT ALL ON *.* TO A") + node.query("CREATE USER B") + node.query("GRANT ALL ON *.* TO B") + + responses_A = list() + responses_B = list() + for _ in range(100): + responses_A.append(node.get_query_request(TEST_QUERY_A, user="A")) + responses_B.append(node.get_query_request(TEST_QUERY_B, user="B")) + + overcommited_killed = False + for response in responses_A: + err = response.get_error() + if "MEMORY_LIMIT_EXCEEDED" in err: + overcommited_killed = True + for response in responses_B: + response.get_answer_and_error() + + assert overcommited_killed, "no overcommited task was killed" + + node.query("DROP USER IF EXISTS A") + node.query("DROP USER IF EXISTS B") From 3e2e422ddd02d9eb2e7b3f92d1a8d4f9fe96c6e0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 8 Feb 2022 12:21:16 +0000 Subject: [PATCH 041/164] refactoring --- src/Interpreters/InterpreterCreateQuery.cpp | 64 +++++++++++---------- src/Interpreters/InterpreterCreateQuery.h | 2 + 2 files changed, 36 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 535b9c07bc6..5918a094db7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -9,10 +9,10 @@ #include #include #include -#include #include #include +#include #include #include @@ -719,6 +719,36 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat } } +String InterpreterCreateQuery::getTableEngineName(DefaultTableEngine default_table_engine) +{ + switch (default_table_engine) + { + case DefaultTableEngine::Log: + return "Log"; + + case DefaultTableEngine::StripeLog: + return "StripeLog"; + + case DefaultTableEngine::MergeTree: + return "MergeTree"; + + case DefaultTableEngine::ReplacingMergeTree: + return "ReplacingMergeTree"; + + case DefaultTableEngine::ReplicatedMergeTree: + return "ReplicatedMergeTree"; + + case DefaultTableEngine::ReplicatedReplacingMergeTree: + return "ReplicatedReplacingMergeTree"; + + case DefaultTableEngine::Memory: + return "Memory"; + + default: + throw Exception("default_table_engine is set to unknown value", ErrorCodes::LOGICAL_ERROR); + } +} + void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { if (create.as_table_function) @@ -791,38 +821,12 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (getContext()->getSettingsRef().default_table_engine.value != DefaultTableEngine::None) { - auto default_table_engine = getContext()->getSettingsRef().default_table_engine.value; - String default_table_engine_name; - switch (default_table_engine) - { - case DefaultTableEngine::Log: - default_table_engine_name = "Log"; - break; - case DefaultTableEngine::StripeLog: - default_table_engine_name = "StripeLog"; - break; - case DefaultTableEngine::MergeTree: - default_table_engine_name = "MergeTree"; - break; - case DefaultTableEngine::ReplacingMergeTree: - default_table_engine_name = "ReplacingMergeTree"; - break; - case DefaultTableEngine::ReplicatedMergeTree: - default_table_engine_name = "ReplicatedMergeTree"; - break; - case DefaultTableEngine::ReplicatedReplacingMergeTree: - default_table_engine_name = "ReplicatedReplacingMergeTree"; - break; - case DefaultTableEngine::Memory: - default_table_engine_name = "Memory"; - break; - default: - throw Exception("default_table_engine is set to unknown value", ErrorCodes::LOGICAL_ERROR); - } + if (!create.storage->engine) { auto engine_ast = std::make_shared(); - engine_ast->name = default_table_engine_name; + auto default_table_engine = getContext()->getSettingsRef().default_table_engine.value; + engine_ast->name = getTableEngineName(default_table_engine); engine_ast->no_empty_args = true; create.storage->set(create.storage->engine, engine_ast); } diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 03c4b4ae1b6..ec4c41825da 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -81,6 +82,7 @@ private: /// Calculate list of columns, constraints, indices, etc... of table. Rewrite query in canonical way. TableProperties getTablePropertiesAndNormalizeCreateQuery(ASTCreateQuery & create) const; void validateTableStructure(const ASTCreateQuery & create, const TableProperties & properties) const; + static String getTableEngineName(DefaultTableEngine default_table_engine); void setEngine(ASTCreateQuery & create) const; AccessRightsElements getRequiredAccess() const; From d2d47b9595d68d2b4b9b13880612de48496d9768 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 8 Feb 2022 16:27:33 +0000 Subject: [PATCH 042/164] Fixing build. --- src/Processors/Formats/Impl/TSKVRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index 8a56c2ed5c7..f63d6fa9c46 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes TSKVRowInputFormat::TSKVRowInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(std::move(header_), in_, std::move(params_)), format_settings(format_settings_), name_map(header_.columns()) + : IRowInputFormat(std::move(header_), in_, std::move(params_)), format_settings(format_settings_), name_map(getPort().getHeader().columns()) { const auto & sample_block = getPort().getHeader(); size_t num_columns = sample_block.columns(); From 8d310e4ac360d448c215a639431a14a55f120f95 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Tue, 8 Feb 2022 19:23:40 +0100 Subject: [PATCH 043/164] AsynchronousInserts -> SystemAsynchronousInserts --- src/Storages/System/StorageSystemAsynchronousInserts.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemAsynchronousInserts.h b/src/Storages/System/StorageSystemAsynchronousInserts.h index 79f19ec3d97..d25217006db 100644 --- a/src/Storages/System/StorageSystemAsynchronousInserts.h +++ b/src/Storages/System/StorageSystemAsynchronousInserts.h @@ -14,7 +14,7 @@ class StorageSystemAsynchronousInserts final : public IStorageSystemOneBlock { public: - std::string getName() const override { return "AsynchronousInserts"; } + std::string getName() const override { return "SystemAsynchronousInserts"; } static NamesAndTypesList getNamesAndTypes(); protected: From 742620209ae98cc02fd50a9448869b636fa25c0b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Feb 2022 03:50:50 +0300 Subject: [PATCH 044/164] static-files-disk-uploader: add mode to create symlinks --- .../static-files-disk-uploader.cpp | 40 +++++++++++-------- src/Storages/HDFS/StorageHDFSCluster.cpp | 16 +------- 2 files changed, 26 insertions(+), 30 deletions(-) diff --git a/programs/static-files-disk-uploader/static-files-disk-uploader.cpp b/programs/static-files-disk-uploader/static-files-disk-uploader.cpp index ad3a3090de6..6ce84f823e9 100644 --- a/programs/static-files-disk-uploader/static-files-disk-uploader.cpp +++ b/programs/static-files-disk-uploader/static-files-disk-uploader.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes * If test-mode option is added, files will be put by given url via PUT request. */ -void processFile(const fs::path & file_path, const fs::path & dst_path, bool test_mode, WriteBuffer & metadata_buf) +void processFile(const fs::path & file_path, const fs::path & dst_path, bool test_mode, bool link, WriteBuffer & metadata_buf) { String remote_path; RE2::FullMatch(file_path.string(), EXTRACT_PATH_PATTERN, &remote_path); @@ -52,22 +52,29 @@ void processFile(const fs::path & file_path, const fs::path & dst_path, bool tes auto dst_file_path = fs::path(dst_path) / remote_path; - auto src_buf = createReadBufferFromFileBase(file_path, {}, fs::file_size(file_path)); - std::shared_ptr dst_buf; - - /// test mode for integration tests. - if (test_mode) - dst_buf = std::make_shared(Poco::URI(dst_file_path), Poco::Net::HTTPRequest::HTTP_PUT); + if (link) + { + fs::create_symlink(file_path, dst_file_path); + } else - dst_buf = std::make_shared(dst_file_path); + { + auto src_buf = createReadBufferFromFileBase(file_path, {}, fs::file_size(file_path)); + std::shared_ptr dst_buf; - copyData(*src_buf, *dst_buf); - dst_buf->next(); - dst_buf->finalize(); + /// test mode for integration tests. + if (test_mode) + dst_buf = std::make_shared(Poco::URI(dst_file_path), Poco::Net::HTTPRequest::HTTP_PUT); + else + dst_buf = std::make_shared(dst_file_path); + + copyData(*src_buf, *dst_buf); + dst_buf->next(); + dst_buf->finalize(); + } }; -void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_mode) +void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_mode, bool link) { std::cerr << "Data path: " << data_path << ", destination path: " << dst_path << std::endl; @@ -94,7 +101,7 @@ void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_ { if (dir_it->is_directory()) { - processFile(dir_it->path(), dst_path, test_mode, *root_meta); + processFile(dir_it->path(), dst_path, test_mode, link, *root_meta); String directory_prefix; RE2::FullMatch(dir_it->path().string(), EXTRACT_PATH_PATTERN, &directory_prefix); @@ -115,14 +122,14 @@ void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_ fs::directory_iterator files_end; for (fs::directory_iterator file_it(dir_it->path()); file_it != files_end; ++file_it) - processFile(file_it->path(), dst_path, test_mode, *directory_meta); + processFile(file_it->path(), dst_path, test_mode, link, *directory_meta); directory_meta->next(); directory_meta->finalize(); } else { - processFile(dir_it->path(), dst_path, test_mode, *root_meta); + processFile(dir_it->path(), dst_path, test_mode, link, *root_meta); } } root_meta->next(); @@ -141,6 +148,7 @@ try ("help,h", "produce help message") ("metadata-path", po::value(), "Metadata path (select data_paths from system.tables where name='table_name'") ("test-mode", "Use test mode, which will put data on given url via PUT") + ("link", "Create symlinks instead of copying") ("url", po::value(), "Web server url for test mode") ("output-dir", po::value(), "Directory to put files in non-test mode"); @@ -186,7 +194,7 @@ try root_path = fs::current_path(); } - processTableFiles(fs_path, root_path, test_mode); + processTableFiles(fs_path, root_path, test_mode, options.count("link")); return 0; } diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index dfe1ea6ffd3..229837c0e17 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -2,17 +2,9 @@ #if USE_HDFS -#include -#include #include #include -#include -#include -#include #include -#include -#include -#include #include #include #include @@ -21,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -29,16 +20,13 @@ #include #include #include -#include -#include #include -#include -#include -#include + namespace DB { + StorageHDFSCluster::StorageHDFSCluster( String cluster_name_, const String & uri_, From 5bef470615d1a796c17b8eaa0b6a89e020024701 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Wed, 9 Feb 2022 07:48:50 +0100 Subject: [PATCH 045/164] Fix test --- .../0_stateless/02117_show_create_table_system.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 82faf3b21ed..678fe35fd96 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -1,5 +1,5 @@ CREATE TABLE system.aggregate_function_combinators\n(\n `name` String,\n `is_internal` UInt8\n)\nENGINE = SystemAggregateFunctionCombinators()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.asynchronous_inserts\n(\n `query` String,\n `database` String,\n `table` String,\n `format` String,\n `first_update` DateTime64(6),\n `last_update` DateTime64(6),\n `total_bytes` UInt64,\n `entries.query_id` Array(String),\n `entries.bytes` Array(UInt64),\n `entries.finished` Array(UInt8),\n `entries.exception` Array(String)\n)\nENGINE = AsynchronousInserts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.asynchronous_inserts\n(\n `query` String,\n `database` String,\n `table` String,\n `format` String,\n `first_update` DateTime64(6),\n `last_update` DateTime64(6),\n `total_bytes` UInt64,\n `entries.query_id` Array(String),\n `entries.bytes` Array(UInt64),\n `entries.finished` Array(UInt8),\n `entries.exception` Array(String)\n)\nENGINE = SystemAsynchronousInserts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.asynchronous_metrics\n(\n `metric` String,\n `value` Float64\n)\nENGINE = SystemAsynchronousMetrics()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.build_options\n(\n `name` String,\n `value` String\n)\nENGINE = SystemBuildOptions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.clusters\n(\n `cluster` String,\n `shard_num` UInt32,\n `shard_weight` UInt32,\n `replica_num` UInt32,\n `host_name` String,\n `host_address` String,\n `port` UInt16,\n `is_local` UInt8,\n `user` String,\n `default_database` String,\n `errors_count` UInt32,\n `slowdowns_count` UInt32,\n `estimated_recovery_time` UInt32\n)\nENGINE = SystemClusters()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' From 881b5ebbbbbd679e3640a441d88563a8da551c18 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 9 Feb 2022 15:53:24 +0300 Subject: [PATCH 046/164] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cd29ef959ad..8df167be453 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -482,7 +482,7 @@ class IColumn; M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ - M(Bool, allow_experimental_projection_optimization, false, "Enable projection optimization when processing SELECT queries", 0) \ + M(Bool, allow_experimental_projection_optimization, true, "Enable projection optimization when processing SELECT queries", 0) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, insert_null_as_default, true, "Insert DEFAULT values instead of NULL in INSERT SELECT (UNION ALL)", 0) \ From 43dc769022fe78d9c8450b858870bad766f9f06b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 9 Feb 2022 14:19:07 +0100 Subject: [PATCH 047/164] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cd29ef959ad..01111752af3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -547,7 +547,7 @@ class IColumn; M(Int64, remote_fs_read_max_backoff_ms, 10000, "Max wait time when trying to read data for remote disk", 0) \ M(Int64, remote_fs_read_backoff_max_tries, 5, "Max attempts to read with backoff", 0) \ \ - M(UInt64, http_max_tries, 1, "Max attempts to read via http.", 0) \ + M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \ M(UInt64, http_retry_initial_backoff_ms, 100, "Min milliseconds for backoff, when retrying read via http", 0) \ M(UInt64, http_retry_max_backoff_ms, 10000, "Max milliseconds for backoff, when retrying read via http", 0) \ \ From 8ea5aab7aa50aa615de0f2827011a00ceae87b5e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 9 Feb 2022 17:02:44 +0300 Subject: [PATCH 048/164] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8df167be453..113cee54e8d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -482,7 +482,6 @@ class IColumn; M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ - M(Bool, allow_experimental_projection_optimization, true, "Enable projection optimization when processing SELECT queries", 0) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, insert_null_as_default, true, "Insert DEFAULT values instead of NULL in INSERT SELECT (UNION ALL)", 0) \ @@ -582,6 +581,7 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ + MAKE_OBSOLETE(M, Bool, allow_experimental_projection_optimization, true) \ /** The section above is for obsolete settings. Do not add anything there. */ From ca0d78295a7ce236f0eb92c4f5383481e26a3083 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 9 Feb 2022 14:19:50 -0400 Subject: [PATCH 049/164] test for #13907 toColumnTypeName_toLowCardinality_const --- .../02210_toColumnTypeName_toLowCardinality_const.reference | 1 + .../02210_toColumnTypeName_toLowCardinality_const.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference create mode 100644 tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.sql diff --git a/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference b/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference new file mode 100644 index 00000000000..2ac2f690f1b --- /dev/null +++ b/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference @@ -0,0 +1 @@ +Const(UInt8) diff --git a/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.sql b/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.sql new file mode 100644 index 00000000000..a71c3f30604 --- /dev/null +++ b/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.sql @@ -0,0 +1 @@ +SELECT toColumnTypeName(toLowCardinality(1)); From d19ee89d9d99981b6a3c4098f167ce11b928c709 Mon Sep 17 00:00:00 2001 From: Geoff Genz Date: Wed, 9 Feb 2022 12:17:23 -0700 Subject: [PATCH 050/164] Remove invalid IOS setting for RocksDB CMAKE to fix Apple M1 build --- contrib/rocksdb-cmake/CMakeLists.txt | 5 ----- 1 file changed, 5 deletions(-) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index c35009ba10a..529d7f0c4e3 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -127,11 +127,6 @@ endif() if(CMAKE_SYSTEM_NAME MATCHES "Darwin") add_definitions(-DOS_MACOSX) - if(CMAKE_SYSTEM_PROCESSOR MATCHES arm) - add_definitions(-DIOS_CROSS_COMPILE -DROCKSDB_LITE) - # no debug info for IOS, that will make our library big - add_definitions(-DNDEBUG) - endif() elseif(CMAKE_SYSTEM_NAME MATCHES "Linux") add_definitions(-DOS_LINUX) elseif(CMAKE_SYSTEM_NAME MATCHES "SunOS") From 45d903b3b56215fe68f76a2da1a95cdfd27e7acf Mon Sep 17 00:00:00 2001 From: Rajkumar Date: Wed, 9 Feb 2022 11:28:08 -0800 Subject: [PATCH 051/164] clang-tidy reported divide by zero exception --- src/AggregateFunctions/ReservoirSampler.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index 1d7529ee8e1..5f7ac13d908 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -239,6 +239,7 @@ private: UInt64 genRandom(size_t lim) { + assert(lim > 0); /// With a large number of values, we will generate random numbers several times slower. if (lim <= static_cast(rng.max())) return static_cast(rng()) % static_cast(lim); From b6a1a13327b644a353d7cffc24e5c21a6ba3fcee Mon Sep 17 00:00:00 2001 From: Rajkumar Date: Wed, 9 Feb 2022 11:36:31 -0800 Subject: [PATCH 052/164] clang-tidy reported nested_column already moved to data --- src/Columns/ColumnArray.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index c18887b7a13..c4d75fed129 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -50,12 +50,12 @@ ColumnArray::ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && if (!offsets_concrete) throw Exception("offsets_column must be a ColumnUInt64", ErrorCodes::LOGICAL_ERROR); - if (!offsets_concrete->empty() && nested_column) + if (!offsets_concrete->empty() && data) { Offset last_offset = offsets_concrete->getData().back(); /// This will also prevent possible overflow in offset. - if (nested_column->size() != last_offset) + if (data->size() != last_offset) throw Exception("offsets_column has data inconsistent with nested_column", ErrorCodes::LOGICAL_ERROR); } From 1f6b65d39d0f8d47ae4c9e677aab04e04c86f654 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 6 Feb 2022 23:10:41 +0700 Subject: [PATCH 053/164] gRPC: Split compression_type field into input_compression_type and output_compression_type. Make the previous field obsolete. --- src/Server/GRPCServer.cpp | 33 +++++++++++++------- src/Server/grpc_protos/clickhouse_grpc.proto | 19 +++++++---- tests/integration/test_grpc_protocol/test.py | 12 +++---- 3 files changed, 40 insertions(+), 24 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 8aa729b8883..9194e087e00 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -628,10 +628,11 @@ namespace ASTInsertQuery * insert_query = nullptr; String input_format; String input_data_delimiter; + CompressionMethod input_compression_method = CompressionMethod::None; PODArray output; String output_format; - CompressionMethod compression_method = CompressionMethod::None; - int compression_level = 0; + CompressionMethod output_compression_method = CompressionMethod::None; + int output_compression_level = 0; uint64_t interactive_delay = 100000; bool send_exception_with_stacktrace = true; @@ -852,8 +853,16 @@ namespace output_format = query_context->getDefaultFormat(); /// Choose compression. - compression_method = chooseCompressionMethod("", query_info.compression_type()); - compression_level = query_info.compression_level(); + String input_compression_method_str = query_info.input_compression_type(); + if (input_compression_method_str.empty()) + input_compression_method_str = query_info.obsolete_compression_type(); + input_compression_method = chooseCompressionMethod("", input_compression_method_str); + + String output_compression_method_str = query_info.output_compression_type(); + if (output_compression_method_str.empty()) + output_compression_method_str = query_info.obsolete_compression_type(); + output_compression_method = chooseCompressionMethod("", output_compression_method_str); + output_compression_level = query_info.output_compression_level(); /// Set callback to create and fill external tables query_context->setExternalTablesInitializer([this] (ContextPtr context) @@ -984,7 +993,7 @@ namespace return {nullptr, 0}; /// no more input data }); - read_buffer = wrapReadBufferWithCompressionMethod(std::move(read_buffer), compression_method); + read_buffer = wrapReadBufferWithCompressionMethod(std::move(read_buffer), input_compression_method); assert(!pipeline); auto source = query_context->getInputFormat( @@ -1112,13 +1121,13 @@ namespace if (io.pipeline.pulling()) header = io.pipeline.getHeader(); - if (compression_method != CompressionMethod::None) + if (output_compression_method != CompressionMethod::None) output.resize(DBMS_DEFAULT_BUFFER_SIZE); /// Must have enough space for compressed data. write_buffer = std::make_unique>>(output); nested_write_buffer = static_cast> *>(write_buffer.get()); - if (compression_method != CompressionMethod::None) + if (output_compression_method != CompressionMethod::None) { - write_buffer = wrapWriteBufferWithCompressionMethod(std::move(write_buffer), compression_method, compression_level); + write_buffer = wrapWriteBufferWithCompressionMethod(std::move(write_buffer), output_compression_method, output_compression_level); compressing_write_buffer = write_buffer.get(); } @@ -1414,10 +1423,10 @@ namespace return; PODArray memory; - if (compression_method != CompressionMethod::None) + if (output_compression_method != CompressionMethod::None) memory.resize(DBMS_DEFAULT_BUFFER_SIZE); /// Must have enough space for compressed data. std::unique_ptr buf = std::make_unique>>(memory); - buf = wrapWriteBufferWithCompressionMethod(std::move(buf), compression_method, compression_level); + buf = wrapWriteBufferWithCompressionMethod(std::move(buf), output_compression_method, output_compression_level); auto format = query_context->getOutputFormat(output_format, *buf, totals); format->write(materializeBlock(totals)); format->finalize(); @@ -1432,10 +1441,10 @@ namespace return; PODArray memory; - if (compression_method != CompressionMethod::None) + if (output_compression_method != CompressionMethod::None) memory.resize(DBMS_DEFAULT_BUFFER_SIZE); /// Must have enough space for compressed data. std::unique_ptr buf = std::make_unique>>(memory); - buf = wrapWriteBufferWithCompressionMethod(std::move(buf), compression_method, compression_level); + buf = wrapWriteBufferWithCompressionMethod(std::move(buf), output_compression_method, output_compression_level); auto format = query_context->getOutputFormat(output_format, *buf, extremes); format->write(materializeBlock(extremes)); format->finalize(); diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index c86c74535c5..b231ad91a32 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -106,12 +106,16 @@ message QueryInfo { /// If not set the compression settings from the configuration file will be used. Compression result_compression = 17; - // Compression type for `input_data`, `output_data`, `totals` and `extremes`. + // Compression type for `input_data`. // Supported compression types: none, gzip(gz), deflate, brotli(br), lzma(xz), zstd(zst), lz4, bz2. - // When used for `input_data` the client is responsible to compress data before putting it into `input_data`. - // When used for `output_data` or `totals` or `extremes` the client receives compressed data and should decompress it by itself. - // In the latter case consider to specify also `compression_level`. - string compression_type = 18; + // The client is responsible to compress data before putting it into `input_data`. + string input_compression_type = 20; + + // Compression type for `output_data`, `totals` and `extremes`. + // Supported compression types: none, gzip(gz), deflate, brotli(br), lzma(xz), zstd(zst), lz4, bz2. + // The client receives compressed data and should decompress it by itself. + // Consider also setting `output_compression_level`. + string output_compression_type = 21; // Compression level. // WARNING: If it's not specified the compression level is set to zero by default which might be not the best choice for some compression types (see below). @@ -123,7 +127,10 @@ message QueryInfo { // zstd: 1..22; 3 is recommended by default (compression level 0 also means 3) // lz4: 0..16; values < 0 mean fast acceleration // bz2: 1..9 - int32 compression_level = 19; + int32 output_compression_level = 19; + + /// Obsolete fields, should not be used in new code. + string obsolete_compression_type = 18; } enum LogsLevel { diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index b6968575883..58ffd04111f 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -382,13 +382,13 @@ def test_result_compression(): assert result.output == (b'0\n')*1000000 def test_compressed_output(): - query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", compression_type="lz4") + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", output_compression_type="lz4") stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) result = stub.ExecuteQuery(query_info) assert lz4.frame.decompress(result.output) == (b'0\n')*1000 def test_compressed_output_streaming(): - query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(100000)", compression_type="lz4") + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(100000)", output_compression_type="lz4") stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) d_context = lz4.frame.create_decompression_context() data = b'' @@ -398,7 +398,7 @@ def test_compressed_output_streaming(): assert data == (b'0\n')*100000 def test_compressed_output_gzip(): - query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", compression_type="gzip", compression_level=6) + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", output_compression_type="gzip", output_compression_level=6) stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) result = stub.ExecuteQuery(query_info) assert gzip.decompress(result.output) == (b'0\n')*1000 @@ -407,10 +407,10 @@ def test_compressed_totals_and_extremes(): query("CREATE TABLE t (x UInt8, y UInt8) ENGINE = Memory") query("INSERT INTO t VALUES (1, 2), (2, 4), (3, 2), (3, 3), (3, 4)") stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) - query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT sum(x), y FROM t GROUP BY y WITH TOTALS", compression_type="lz4") + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT sum(x), y FROM t GROUP BY y WITH TOTALS", output_compression_type="lz4") result = stub.ExecuteQuery(query_info) assert lz4.frame.decompress(result.totals) == b'12\t0\n' - query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT x, y FROM t", settings={"extremes": "1"}, compression_type="lz4") + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT x, y FROM t", settings={"extremes": "1"}, output_compression_type="lz4") result = stub.ExecuteQuery(query_info) assert lz4.frame.decompress(result.extremes) == b'1\t2\n3\t4\n' @@ -423,7 +423,7 @@ def test_compressed_insert_query_streaming(): d2 = data[sz1:sz1+sz2] d3 = data[sz1+sz2:] def send_query_info(): - yield clickhouse_grpc_pb2.QueryInfo(query="INSERT INTO t VALUES", input_data=d1, compression_type="lz4", next_query_info=True) + yield clickhouse_grpc_pb2.QueryInfo(query="INSERT INTO t VALUES", input_data=d1, input_compression_type="lz4", next_query_info=True) yield clickhouse_grpc_pb2.QueryInfo(input_data=d2, next_query_info=True) yield clickhouse_grpc_pb2.QueryInfo(input_data=d3) stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) From 1341b4b4de5b8443760ba4b6dd1f2b4a9aa97445 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 7 Feb 2022 01:33:31 +0700 Subject: [PATCH 054/164] Rename QueryInfo's field result_compression -> transport_compression_type and change its type for better consistency. Make the previous field obsolete. --- programs/server/config.xml | 9 +- src/Server/GRPCServer.cpp | 175 +++++++++++-------- src/Server/grpc_protos/clickhouse_grpc.proto | 45 +++-- tests/integration/test_grpc_protocol/test.py | 14 +- 4 files changed, 142 insertions(+), 101 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index ce0c54f6730..def64607caf 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -217,13 +217,12 @@ /path/to/ssl_ca_cert_file - - deflate + none - - medium + + 0 -1 diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 9194e087e00..10bbce24913 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -51,6 +51,7 @@ using GRPCQueryInfo = clickhouse::grpc::QueryInfo; using GRPCResult = clickhouse::grpc::Result; using GRPCException = clickhouse::grpc::Exception; using GRPCProgress = clickhouse::grpc::Progress; +using GRPCObsoleteTransportCompression = clickhouse::grpc::ObsoleteTransportCompression; namespace DB { @@ -101,62 +102,6 @@ namespace }); } - grpc_compression_algorithm parseCompressionAlgorithm(const String & str) - { - if (str == "none") - return GRPC_COMPRESS_NONE; - else if (str == "deflate") - return GRPC_COMPRESS_DEFLATE; - else if (str == "gzip") - return GRPC_COMPRESS_GZIP; - else if (str == "stream_gzip") - return GRPC_COMPRESS_STREAM_GZIP; - else - throw Exception("Unknown compression algorithm: '" + str + "'", ErrorCodes::INVALID_CONFIG_PARAMETER); - } - - grpc_compression_level parseCompressionLevel(const String & str) - { - if (str == "none") - return GRPC_COMPRESS_LEVEL_NONE; - else if (str == "low") - return GRPC_COMPRESS_LEVEL_LOW; - else if (str == "medium") - return GRPC_COMPRESS_LEVEL_MED; - else if (str == "high") - return GRPC_COMPRESS_LEVEL_HIGH; - else - throw Exception("Unknown compression level: '" + str + "'", ErrorCodes::INVALID_CONFIG_PARAMETER); - } - - grpc_compression_algorithm convertCompressionAlgorithm(const ::clickhouse::grpc::CompressionAlgorithm & algorithm) - { - if (algorithm == ::clickhouse::grpc::NO_COMPRESSION) - return GRPC_COMPRESS_NONE; - else if (algorithm == ::clickhouse::grpc::DEFLATE) - return GRPC_COMPRESS_DEFLATE; - else if (algorithm == ::clickhouse::grpc::GZIP) - return GRPC_COMPRESS_GZIP; - else if (algorithm == ::clickhouse::grpc::STREAM_GZIP) - return GRPC_COMPRESS_STREAM_GZIP; - else - throw Exception("Unknown compression algorithm: '" + ::clickhouse::grpc::CompressionAlgorithm_Name(algorithm) + "'", ErrorCodes::INVALID_GRPC_QUERY_INFO); - } - - grpc_compression_level convertCompressionLevel(const ::clickhouse::grpc::CompressionLevel & level) - { - if (level == ::clickhouse::grpc::COMPRESSION_NONE) - return GRPC_COMPRESS_LEVEL_NONE; - else if (level == ::clickhouse::grpc::COMPRESSION_LOW) - return GRPC_COMPRESS_LEVEL_LOW; - else if (level == ::clickhouse::grpc::COMPRESSION_MEDIUM) - return GRPC_COMPRESS_LEVEL_MED; - else if (level == ::clickhouse::grpc::COMPRESSION_HIGH) - return GRPC_COMPRESS_LEVEL_HIGH; - else - throw Exception("Unknown compression level: '" + ::clickhouse::grpc::CompressionLevel_Name(level) + "'", ErrorCodes::INVALID_GRPC_QUERY_INFO); - } - /// Gets file's contents as a string, throws an exception if failed. String readFile(const String & filepath) { @@ -193,6 +138,102 @@ namespace return grpc::InsecureServerCredentials(); } + /// Transport compression makes gRPC library to compress packed Result messages before sending them through network. + struct TransportCompression + { + grpc_compression_algorithm algorithm; + grpc_compression_level level; + + /// Extracts the settings of transport compression from a query info if possible. + static std::optional fromQueryInfo(const GRPCQueryInfo & query_info) + { + TransportCompression res; + if (!query_info.transport_compression_type().empty()) + { + res.setAlgorithm(query_info.transport_compression_type(), ErrorCodes::INVALID_GRPC_QUERY_INFO); + res.setLevel(query_info.transport_compression_level(), ErrorCodes::INVALID_GRPC_QUERY_INFO); + return res; + } + + if (query_info.has_obsolete_result_compression()) + { + switch (query_info.obsolete_result_compression().algorithm()) + { + case GRPCObsoleteTransportCompression::NO_COMPRESSION: res.algorithm = GRPC_COMPRESS_NONE; break; + case GRPCObsoleteTransportCompression::DEFLATE: res.algorithm = GRPC_COMPRESS_DEFLATE; break; + case GRPCObsoleteTransportCompression::GZIP: res.algorithm = GRPC_COMPRESS_GZIP; break; + case GRPCObsoleteTransportCompression::STREAM_GZIP: res.algorithm = GRPC_COMPRESS_STREAM_GZIP; break; + default: throw Exception(ErrorCodes::INVALID_GRPC_QUERY_INFO, "Unknown compression algorithm: {}", GRPCObsoleteTransportCompression::CompressionAlgorithm_Name(query_info.obsolete_result_compression().algorithm())); + } + + switch (query_info.obsolete_result_compression().level()) + { + case GRPCObsoleteTransportCompression::COMPRESSION_NONE: res.level = GRPC_COMPRESS_LEVEL_NONE; break; + case GRPCObsoleteTransportCompression::COMPRESSION_LOW: res.level = GRPC_COMPRESS_LEVEL_LOW; break; + case GRPCObsoleteTransportCompression::COMPRESSION_MEDIUM: res.level = GRPC_COMPRESS_LEVEL_MED; break; + case GRPCObsoleteTransportCompression::COMPRESSION_HIGH: res.level = GRPC_COMPRESS_LEVEL_HIGH; break; + default: throw Exception(ErrorCodes::INVALID_GRPC_QUERY_INFO, "Unknown compression level: {}", GRPCObsoleteTransportCompression::CompressionLevel_Name(query_info.obsolete_result_compression().level())); + } + return res; + } + + return std::nullopt; + } + + /// Extracts the settings of transport compression from the server configuration. + static TransportCompression fromConfiguration(const Poco::Util::AbstractConfiguration & config) + { + TransportCompression res; + if (config.has("grpc.transport_compression_type")) + { + res.setAlgorithm(config.getString("grpc.transport_compression_type"), ErrorCodes::INVALID_CONFIG_PARAMETER); + res.setLevel(config.getInt("grpc.transport_compression_level", 0), ErrorCodes::INVALID_CONFIG_PARAMETER); + } + else + { + res.setAlgorithm(config.getString("grpc.compression", "none"), ErrorCodes::INVALID_CONFIG_PARAMETER); + res.setLevel(config.getString("grpc.compression_level", "none"), ErrorCodes::INVALID_CONFIG_PARAMETER); + } + return res; + } + + private: + void setAlgorithm(const String & str, int error_code) + { + if (str == "none") + algorithm = GRPC_COMPRESS_NONE; + else if (str == "deflate") + algorithm = GRPC_COMPRESS_DEFLATE; + else if (str == "gzip") + algorithm = GRPC_COMPRESS_GZIP; + else if (str == "stream_gzip") + algorithm = GRPC_COMPRESS_STREAM_GZIP; + else + throw Exception(error_code, "Unknown compression algorithm: '{}'", str); + } + + void setLevel(const String & str, int error_code) + { + if (str == "none") + level = GRPC_COMPRESS_LEVEL_NONE; + else if (str == "low") + level = GRPC_COMPRESS_LEVEL_LOW; + else if (str == "medium") + level = GRPC_COMPRESS_LEVEL_MED; + else if (str == "high") + level = GRPC_COMPRESS_LEVEL_HIGH; + else + throw Exception(error_code, "Unknown compression level: '{}'", str); + } + + void setLevel(int level_, int error_code) + { + if (0 <= level_ && level_ < GRPC_COMPRESS_LEVEL_COUNT) + level = static_cast(level_); + else + throw Exception(error_code, "Compression level {} is out of range 0..{}", level_, GRPC_COMPRESS_LEVEL_COUNT - 1); + } + }; /// Gets session's timeout from query info or from the server config. std::chrono::steady_clock::duration getSessionTimeout(const GRPCQueryInfo & query_info, const Poco::Util::AbstractConfiguration & config) @@ -293,15 +334,10 @@ namespace return std::nullopt; } - void setResultCompression(grpc_compression_algorithm algorithm, grpc_compression_level level) + void setTransportCompression(const TransportCompression & transport_compression) { - grpc_context.set_compression_algorithm(algorithm); - grpc_context.set_compression_level(level); - } - - void setResultCompression(const ::clickhouse::grpc::Compression & compression) - { - setResultCompression(convertCompressionAlgorithm(compression.algorithm()), convertCompressionLevel(compression.level())); + grpc_context.set_compression_algorithm(transport_compression.algorithm); + grpc_context.set_compression_level(transport_compression.level); } protected: @@ -816,9 +852,9 @@ namespace if (!query_info.database().empty()) query_context->setCurrentDatabase(query_info.database()); - /// Apply compression settings for this call. - if (query_info.has_result_compression()) - responder->setResultCompression(query_info.result_compression()); + /// Apply transport compression for this call. + if (auto transport_compression = TransportCompression::fromQueryInfo(query_info)) + responder->setTransportCompression(*transport_compression); /// The interactive delay will be used to show progress. interactive_delay = settings.interactive_delay; @@ -1781,8 +1817,9 @@ void GRPCServer::start() builder.RegisterService(&grpc_service); builder.SetMaxSendMessageSize(iserver.config().getInt("grpc.max_send_message_size", -1)); builder.SetMaxReceiveMessageSize(iserver.config().getInt("grpc.max_receive_message_size", -1)); - builder.SetDefaultCompressionAlgorithm(parseCompressionAlgorithm(iserver.config().getString("grpc.compression", "none"))); - builder.SetDefaultCompressionLevel(parseCompressionLevel(iserver.config().getString("grpc.compression_level", "none"))); + auto default_transport_compression = TransportCompression::fromConfiguration(iserver.config()); + builder.SetDefaultCompressionAlgorithm(default_transport_compression.algorithm); + builder.SetDefaultCompressionLevel(default_transport_compression.level); queue = builder.AddCompletionQueue(); grpc_server = builder.BuildAndStart(); diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index b231ad91a32..f596c3b7d6d 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -45,21 +45,19 @@ message ExternalTable { map settings = 5; } -enum CompressionAlgorithm { - NO_COMPRESSION = 0; - DEFLATE = 1; - GZIP = 2; - STREAM_GZIP = 3; -} - -enum CompressionLevel { - COMPRESSION_NONE = 0; - COMPRESSION_LOW = 1; - COMPRESSION_MEDIUM = 2; - COMPRESSION_HIGH = 3; -} - -message Compression { +message ObsoleteTransportCompression { + enum CompressionAlgorithm { + NO_COMPRESSION = 0; + DEFLATE = 1; + GZIP = 2; + STREAM_GZIP = 3; + } + enum CompressionLevel { + COMPRESSION_NONE = 0; + COMPRESSION_LOW = 1; + COMPRESSION_MEDIUM = 2; + COMPRESSION_HIGH = 3; + } CompressionAlgorithm algorithm = 1; CompressionLevel level = 2; } @@ -102,10 +100,6 @@ message QueryInfo { // `next_query_info` is allowed to be set only if a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used. bool next_query_info = 16; - /// Controls how a ClickHouse server will compress query execution results before sending back to the client. - /// If not set the compression settings from the configuration file will be used. - Compression result_compression = 17; - // Compression type for `input_data`. // Supported compression types: none, gzip(gz), deflate, brotli(br), lzma(xz), zstd(zst), lz4, bz2. // The client is responsible to compress data before putting it into `input_data`. @@ -129,7 +123,20 @@ message QueryInfo { // bz2: 1..9 int32 output_compression_level = 19; + // Transport compression is an alternative way to make the server to compress its response. + // This kind of compression implies that instead of compressing just `output` the server will compress whole packed messages of the `Result` type, + // and then gRPC implementation on client side will decompress those messages so client code won't be bothered with decompression. + // Here is a big difference between the transport compression and the compression enabled by setting `output_compression_type` because + // in case of the transport compression the client code receives already decompressed data in `output`. + // If the transport compression is not set here it can still be enabled by the server configuration. + // Supported compression types: none, deflate, gzip, stream_gzip + // Supported compression levels: 0..3 + // WARNING: Don't set `transport_compression` and `output_compression` at the same time because it will make the server to compress its output twice! + string transport_compression_type = 22; + int32 transport_compression_level = 23; + /// Obsolete fields, should not be used in new code. + ObsoleteTransportCompression obsolete_result_compression = 17; string obsolete_compression_type = 18; } diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 58ffd04111f..65ee3cb4261 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -373,14 +373,6 @@ def test_cancel_while_generating_output(): output += result.output assert output == b'0\t0\n1\t0\n2\t0\n3\t0\n' -def test_result_compression(): - query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000000)", - result_compression=clickhouse_grpc_pb2.Compression(algorithm=clickhouse_grpc_pb2.CompressionAlgorithm.GZIP, - level=clickhouse_grpc_pb2.CompressionLevel.COMPRESSION_HIGH)) - stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) - result = stub.ExecuteQuery(query_info) - assert result.output == (b'0\n')*1000000 - def test_compressed_output(): query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", output_compression_type="lz4") stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) @@ -445,6 +437,12 @@ def test_compressed_external_table(): b"4\tDaniel\n"\ b"5\tEthan\n" +def test_transport_compression(): + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000000)", transport_compression_type='gzip', transport_compression_level=3) + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) + result = stub.ExecuteQuery(query_info) + assert result.output == (b'0\n')*1000000 + def test_opentelemetry_context_propagation(): trace_id = "80c190b5-9dc1-4eae-82b9-6c261438c817" parent_span_id = 123 From 77d9cddfec0cf481b787a18ccdba5f2a0489762c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 8 Feb 2022 20:15:56 +0700 Subject: [PATCH 055/164] Replace clickhouse_grpc.proto in a test with symlink. --- .../protos/clickhouse_grpc.proto | 175 +----------------- 1 file changed, 1 insertion(+), 174 deletions(-) mode change 100644 => 120000 tests/integration/test_server_reload/protos/clickhouse_grpc.proto diff --git a/tests/integration/test_server_reload/protos/clickhouse_grpc.proto b/tests/integration/test_server_reload/protos/clickhouse_grpc.proto deleted file mode 100644 index c6cafaf6e40..00000000000 --- a/tests/integration/test_server_reload/protos/clickhouse_grpc.proto +++ /dev/null @@ -1,174 +0,0 @@ -/* This file describes gRPC protocol supported in ClickHouse. - * - * To use this protocol a client should send one or more messages of the QueryInfo type - * and then receive one or more messages of the Result type. - * According to that the service provides four methods for that: - * ExecuteQuery(QueryInfo) returns (Result) - * ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result) - * ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result) - * ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result) - * It's up to the client to choose which method to use. - * For example, ExecuteQueryWithStreamInput() allows the client to add data multiple times - * while executing a query, which is suitable for inserting many rows. - */ - -syntax = "proto3"; - -package clickhouse.grpc; - -message NameAndType { - string name = 1; - string type = 2; -} - -// Describes an external table - a table which will exists only while a query is executing. -message ExternalTable { - // Name of the table. If omitted, "_data" is used. - string name = 1; - - // Columns of the table. Types are required, names can be omitted. If the names are omitted, "_1", "_2", ... is used. - repeated NameAndType columns = 2; - - // Data to insert to the external table. - // If a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used, - // then data for insertion to the same external table can be split between multiple QueryInfos. - bytes data = 3; - - // Format of the data to insert to the external table. - string format = 4; - - // Settings for executing that insertion, applied after QueryInfo.settings. - map settings = 5; -} - -enum CompressionAlgorithm { - NO_COMPRESSION = 0; - DEFLATE = 1; - GZIP = 2; - STREAM_GZIP = 3; -} - -enum CompressionLevel { - COMPRESSION_NONE = 0; - COMPRESSION_LOW = 1; - COMPRESSION_MEDIUM = 2; - COMPRESSION_HIGH = 3; -} - -message Compression { - CompressionAlgorithm algorithm = 1; - CompressionLevel level = 2; -} - -// Information about a query which a client sends to a ClickHouse server. -// The first QueryInfo can set any of the following fields. Extra QueryInfos only add extra data. -// In extra QueryInfos only `input_data`, `external_tables`, `next_query_info` and `cancel` fields can be set. -message QueryInfo { - string query = 1; - string query_id = 2; - map settings = 3; - - // Default database. - string database = 4; - - // Input data, used both as data for INSERT query and as data for the input() function. - bytes input_data = 5; - - // Delimiter for input_data, inserted between input_data from adjacent QueryInfos. - bytes input_data_delimiter = 6; - - // Default output format. If not specified, 'TabSeparated' is used. - string output_format = 7; - - repeated ExternalTable external_tables = 8; - - string user_name = 9; - string password = 10; - string quota = 11; - - // Works exactly like sessions in the HTTP protocol. - string session_id = 12; - bool session_check = 13; - uint32 session_timeout = 14; - - // Set `cancel` to true to stop executing the query. - bool cancel = 15; - - // If true there will be at least one more QueryInfo in the input stream. - // `next_query_info` is allowed to be set only if a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used. - bool next_query_info = 16; - - /// Controls how a ClickHouse server will compress query execution results before sending back to the client. - /// If not set the compression settings from the configuration file will be used. - Compression result_compression = 17; -} - -enum LogsLevel { - LOG_NONE = 0; - LOG_FATAL = 1; - LOG_CRITICAL = 2; - LOG_ERROR = 3; - LOG_WARNING = 4; - LOG_NOTICE = 5; - LOG_INFORMATION = 6; - LOG_DEBUG = 7; - LOG_TRACE = 8; -} - -message LogEntry { - uint32 time = 1; - uint32 time_microseconds = 2; - uint64 thread_id = 3; - string query_id = 4; - LogsLevel level = 5; - string source = 6; - string text = 7; -} - -message Progress { - uint64 read_rows = 1; - uint64 read_bytes = 2; - uint64 total_rows_to_read = 3; - uint64 written_rows = 4; - uint64 written_bytes = 5; -} - -message Stats { - uint64 rows = 1; - uint64 blocks = 2; - uint64 allocated_bytes = 3; - bool applied_limit = 4; - uint64 rows_before_limit = 5; -} - -message Exception { - int32 code = 1; - string name = 2; - string display_text = 3; - string stack_trace = 4; -} - -// Result of execution of a query which is sent back by the ClickHouse server to the client. -message Result { - // Output of the query, represented in the `output_format` or in a format specified in `query`. - bytes output = 1; - bytes totals = 2; - bytes extremes = 3; - - repeated LogEntry logs = 4; - Progress progress = 5; - Stats stats = 6; - - // Set by the ClickHouse server if there was an exception thrown while executing. - Exception exception = 7; - - // Set by the ClickHouse server if executing was cancelled by the `cancel` field in QueryInfo. - bool cancelled = 8; -} - -service ClickHouse { - rpc ExecuteQuery(QueryInfo) returns (Result) {} - rpc ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result) {} - rpc ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result) {} - rpc ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result) {} -} diff --git a/tests/integration/test_server_reload/protos/clickhouse_grpc.proto b/tests/integration/test_server_reload/protos/clickhouse_grpc.proto new file mode 120000 index 00000000000..25d15f11e3b --- /dev/null +++ b/tests/integration/test_server_reload/protos/clickhouse_grpc.proto @@ -0,0 +1 @@ +../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file From d2e1f66c57fd66c785d1cde3a858831299d44802 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 10 Feb 2022 01:01:14 +0000 Subject: [PATCH 056/164] Fix test --- programs/server/Server.cpp | 7 +++++-- .../configs/config.xml | 3 --- .../configs/global_overcommit_tracker.xml | 4 ++++ .../test_global_overcommit_tracker/test.py | 15 +++++++++------ 4 files changed, 18 insertions(+), 11 deletions(-) delete mode 100644 tests/integration/test_global_overcommit_tracker/configs/config.xml create mode 100644 tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ffbb43952cd..583f5a68804 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -921,9 +921,12 @@ if (ThreadFuzzer::instance().isEffective()) total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - UInt64 max_overcommit_wait_time = config->getUInt64("memory_usage_overcommit_max_wait_microseconds", 0); auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); - global_overcommit_tracker->setMaxWaitTime(max_overcommit_wait_time); + if (config->has("global_memory_usage_overcommit_max_wait_microseconds")) + { + UInt64 max_overcommit_wait_time = config->getUInt64("global_memory_usage_overcommit_max_wait_microseconds", 0); + global_overcommit_tracker->setMaxWaitTime(max_overcommit_wait_time); + } total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); // FIXME logging-related things need synchronization -- see the 'Logger * log' saved diff --git a/tests/integration/test_global_overcommit_tracker/configs/config.xml b/tests/integration/test_global_overcommit_tracker/configs/config.xml deleted file mode 100644 index 10c21b03f83..00000000000 --- a/tests/integration/test_global_overcommit_tracker/configs/config.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 5000000 - \ No newline at end of file diff --git a/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml b/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml new file mode 100644 index 00000000000..590759bd15d --- /dev/null +++ b/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml @@ -0,0 +1,4 @@ + + 50000000 + 500 + \ No newline at end of file diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py index e419cb8ad64..229fcd6514b 100644 --- a/tests/integration/test_global_overcommit_tracker/test.py +++ b/tests/integration/test_global_overcommit_tracker/test.py @@ -4,7 +4,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', main_configs=['configs/config.xml']) +node = cluster.add_instance('node', main_configs=['configs/global_overcommit_tracker.xml']) @pytest.fixture(scope='module', autouse=True) def start_cluster(): @@ -14,8 +14,8 @@ def start_cluster(): finally: cluster.shutdown() -TEST_QUERY_A = 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user=1,memory_usage_overcommit_max_wait_microseconds=500' -TEST_QUERY_B = 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user=2,memory_usage_overcommit_max_wait_microseconds=500' +TEST_QUERY_A = 'SELECT number FROM numbers(10000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user=1' +TEST_QUERY_B = 'SELECT number FROM numbers(10000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user=2' def test_overcommited_is_killed(): node.query("CREATE USER A") @@ -31,13 +31,16 @@ def test_overcommited_is_killed(): overcommited_killed = False for response in responses_A: - err = response.get_error() + _, err = response.get_answer_and_error() if "MEMORY_LIMIT_EXCEEDED" in err: overcommited_killed = True + finished = False for response in responses_B: - response.get_answer_and_error() + _, err = response.get_answer_and_error() + if err == "": + finished = True - assert overcommited_killed, "no overcommited task was killed" + assert overcommited_killed and finished, "no overcommited task was killed or all tasks are killed" node.query("DROP USER IF EXISTS A") node.query("DROP USER IF EXISTS B") From bfa96463ca17cec6f4c03bc791a553f6baa03bc3 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Feb 2022 09:23:27 +0300 Subject: [PATCH 057/164] Fix possible error 'file_size: Operation not supported' --- src/Storages/StorageFile.cpp | 3 ++- tests/performance/select_format.xml | 4 ++-- tests/performance/writing_valid_utf8.xml | 2 +- tests/queries/0_stateless/02210_append_to_dev_dull.reference | 0 tests/queries/0_stateless/02210_append_to_dev_dull.sql | 4 ++++ 5 files changed, 9 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02210_append_to_dev_dull.reference create mode 100644 tests/queries/0_stateless/02210_append_to_dev_dull.sql diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index edd5e0447d5..ddc7717567a 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -879,9 +879,10 @@ SinkToStoragePtr StorageFile::write( path = paths.back(); fs::create_directories(fs::path(path).parent_path()); + std::error_code error_code; if (!context->getSettingsRef().engine_file_truncate_on_insert && !is_path_with_globs && !FormatFactory::instance().checkIfFormatSupportAppend(format_name, context, format_settings) && fs::exists(paths.back()) - && fs::file_size(paths.back()) != 0) + && fs::file_size(paths.back(), error_code) != 0 && !error_code) { if (context->getSettingsRef().engine_file_allow_create_multiple_files) { diff --git a/tests/performance/select_format.xml b/tests/performance/select_format.xml index f0114d7a517..982039102d0 100644 --- a/tests/performance/select_format.xml +++ b/tests/performance/select_format.xml @@ -49,8 +49,8 @@ CREATE TABLE IF NOT EXISTS table_{format_slow} ENGINE = File({format_slow}, '/dev/null') AS test.hits CREATE TABLE IF NOT EXISTS table_{format_fast} ENGINE = File({format_fast}, '/dev/null') AS test.hits - INSERT INTO table_{format_slow} SELECT * FROM test.hits LIMIT 10000 SETTINGS engine_file_truncate_on_insert = 1 - INSERT INTO table_{format_fast} SELECT * FROM test.hits LIMIT 100000 SETTINGS engine_file_truncate_on_insert = 1 + INSERT INTO table_{format_slow} SELECT * FROM test.hits LIMIT 10000 + INSERT INTO table_{format_fast} SELECT * FROM test.hits LIMIT 100000 DROP TABLE IF EXISTS table_{format_slow} DROP TABLE IF EXISTS table_{format_fast} diff --git a/tests/performance/writing_valid_utf8.xml b/tests/performance/writing_valid_utf8.xml index 4f6637f7566..d5343144db0 100644 --- a/tests/performance/writing_valid_utf8.xml +++ b/tests/performance/writing_valid_utf8.xml @@ -20,7 +20,7 @@ CREATE TABLE IF NOT EXISTS table_{format} ENGINE = File({format}, '/dev/null') AS SELECT SearchPhrase, ClientIP6, URL, Referer, URLDomain FROM test.hits limit 0 - INSERT INTO table_{format} SELECT SearchPhrase, ClientIP6, URL, Referer, URLDomain FROM test.hits LIMIT 100000 SETTINGS engine_file_truncate_on_insert = 1 + INSERT INTO table_{format} SELECT SearchPhrase, ClientIP6, URL, Referer, URLDomain FROM test.hits LIMIT 100000 DROP TABLE IF EXISTS table_{format} diff --git a/tests/queries/0_stateless/02210_append_to_dev_dull.reference b/tests/queries/0_stateless/02210_append_to_dev_dull.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02210_append_to_dev_dull.sql b/tests/queries/0_stateless/02210_append_to_dev_dull.sql new file mode 100644 index 00000000000..4ae77132972 --- /dev/null +++ b/tests/queries/0_stateless/02210_append_to_dev_dull.sql @@ -0,0 +1,4 @@ +insert into table function file('/dev/null', 'Parquet', 'number UInt64') select * from numbers(10); +insert into table function file('/dev/null', 'ORC', 'number UInt64') select * from numbers(10); +insert into table function file('/dev/null', 'JSON', 'number UInt64') select * from numbers(10); + From c9d27fcea3ab58ec74c2c8d02d7ef8c8e7f0d1ce Mon Sep 17 00:00:00 2001 From: cnmade Date: Thu, 10 Feb 2022 14:45:34 +0800 Subject: [PATCH 058/164] sync translate to zh: Update roadmap.md --- docs/zh/whats-new/roadmap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/whats-new/roadmap.md b/docs/zh/whats-new/roadmap.md index 3cb9dd6fa2f..8e8873c8ee4 100644 --- a/docs/zh/whats-new/roadmap.md +++ b/docs/zh/whats-new/roadmap.md @@ -5,6 +5,6 @@ toc_title: Roadmap # Roadmap {#roadmap} -`2021年Roadmap`已公布供公开讨论查看[这里](https://github.com/ClickHouse/ClickHouse/issues/17623). +`2022年Roadmap`已公布供公开讨论查看 [这里](https://github.com/ClickHouse/ClickHouse/issues/32513). {## [源文章](https://clickhouse.com/docs/en/roadmap/) ##} From 72ffcbbb0563c0819362bbb5b9b39f8be1c7ab4c Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 10 Feb 2022 15:43:39 +0800 Subject: [PATCH 059/164] keeper SnapshotableHashTable clean opt. --- src/Coordination/SnapshotableHashTable.h | 28 +++++++++++------------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index b1d72578530..a837e51a9be 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -36,6 +36,7 @@ private: /// Allows to avoid additional copies in updateValue function size_t snapshot_up_to_size = 0; ArenaWithFreeLists arena; + std::vector snapshot_invalid_iters{100000}; uint64_t approximate_data_size{0}; @@ -175,6 +176,7 @@ public: list_itr->active_in_map = false; auto new_list_itr = list.insert(list.end(), elem); it->getMapped() = new_list_itr; + snapshot_invalid_iters.push_back(list_itr); } else { @@ -197,6 +199,7 @@ public: list_itr->active_in_map = false; list_itr->free_key = true; map.erase(it->getKey()); + snapshot_invalid_iters.push_back(list_itr); } else { @@ -239,6 +242,7 @@ public: auto itr = list.insert(list.end(), elem_copy); it->getMapped() = itr; ret = itr; + snapshot_invalid_iters.push_back(list_itr); } else { @@ -274,23 +278,15 @@ public: void clearOutdatedNodes() { - auto start = list.begin(); - auto end = list.end(); - for (auto itr = start; itr != end;) + for (auto & itr: snapshot_invalid_iters) { - if (!itr->active_in_map) - { - updateDataSize(CLEAR_OUTDATED_NODES, itr->key.size, itr->value.sizeInBytes(), 0); - if (itr->free_key) - arena.free(const_cast(itr->key.data), itr->key.size); - itr = list.erase(itr); - } - else - { - assert(!itr->free_key); - itr++; - } + assert(!itr->active_in_map); + updateDataSize(CLEAR_OUTDATED_NODES, itr->key.size, itr->value.sizeInBytes(), 0); + if (itr->free_key) + arena.free(const_cast(itr->key.data), itr->key.size); + list.erase(itr); } + snapshot_invalid_iters.clear(); } void clear() @@ -300,12 +296,14 @@ public: arena.free(const_cast(itr->key.data), itr->key.size); list.clear(); updateDataSize(CLEAR, 0, 0, 0); + snapshot_invalid_iters.clear(); } void enableSnapshotMode(size_t up_to_size) { snapshot_mode = true; snapshot_up_to_size = up_to_size; + snapshot_invalid_iters.clear(); } void disableSnapshotMode() From 9228b10fa6acde7a75e545fa3d0e64b50f19d1e4 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 10 Feb 2022 08:53:50 +0000 Subject: [PATCH 060/164] refactoring --- src/Interpreters/InterpreterCreateQuery.cpp | 39 +++++++++++---------- 1 file changed, 21 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 5918a094db7..96a494068e5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -2,6 +2,7 @@ #include +#include "Common/Exception.h" #include #include #include @@ -754,14 +755,19 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (create.as_table_function) return; - if (create.storage || create.is_dictionary || create.isView()) - { - if (create.temporary && create.storage && create.storage->engine && create.storage->engine->name != "Memory") - throw Exception(ErrorCodes::INCORRECT_QUERY, - "Temporary tables can only be created with ENGINE = Memory, not {}", create.storage->engine->name); - } + if (create.temporary && create.storage && !create.storage->engine) + throw Exception("Storage is defined without ENGINE for temprorary table.", ErrorCodes::LOGICAL_ERROR); - if (create.temporary && !create.storage) + if (create.temporary && create.storage && create.storage->engine) + { + if (create.storage->engine->name == "Memory") + return; + + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Temporary tables can only be created with ENGINE = Memory, not {}", create.storage->engine->name); + } + + if (create.temporary) { auto engine_ast = std::make_shared(); engine_ast->name = "Memory"; @@ -771,7 +777,8 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const create.set(create.storage, storage_ast); return; } - else if (!create.as_table.empty()) + + if (!create.as_table.empty()) { /// NOTE Getting the structure from the table specified in the AS is done not atomically with the creation of the table. @@ -819,17 +826,13 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (!create.storage) create.set(create.storage, std::make_shared()); - if (getContext()->getSettingsRef().default_table_engine.value != DefaultTableEngine::None) + if (!create.storage->engine && getContext()->getSettingsRef().default_table_engine.value != DefaultTableEngine::None) { - - if (!create.storage->engine) - { - auto engine_ast = std::make_shared(); - auto default_table_engine = getContext()->getSettingsRef().default_table_engine.value; - engine_ast->name = getTableEngineName(default_table_engine); - engine_ast->no_empty_args = true; - create.storage->set(create.storage->engine, engine_ast); - } + auto engine_ast = std::make_shared(); + auto default_table_engine = getContext()->getSettingsRef().default_table_engine.value; + engine_ast->name = getTableEngineName(default_table_engine); + engine_ast->no_empty_args = true; + create.storage->set(create.storage->engine, engine_ast); } } From ea03fec7b64bd14e84f66fd0bdca55f370e87e9c Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 10 Feb 2022 09:00:00 +0000 Subject: [PATCH 061/164] simplify case --- src/Parsers/ParserCreateQuery.cpp | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 0653e1632b8..8c34bb4cbc3 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -580,14 +580,10 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe { // ENGINE can be omitted if default_table_engine is set. // Need to check in Interpreter - if (columns_list) + if (!storage) { - auto columns = columns_list->as(); - if (columns.primary_key) - { - auto storage_ast = std::make_shared(); - storage = storage_ast; - } + auto storage_ast = std::make_shared(); + storage = storage_ast; } } } From 298838f891ce099cef498f22bf91ba44dc56a173 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 10 Feb 2022 00:13:06 +0300 Subject: [PATCH 062/164] avoid unnecessary copying of Settings --- src/Client/ClientBase.cpp | 2 +- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 4 ++-- src/Dictionaries/ExecutablePoolDictionarySource.cpp | 2 +- src/Interpreters/ClusterDiscovery.cpp | 2 +- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 4 ++-- src/Processors/Transforms/getSourceFromASTInsertQuery.cpp | 2 +- src/Server/HTTP/HTTPServerConnection.cpp | 2 +- src/Storages/HDFS/StorageHDFSCluster.cpp | 2 +- src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.cpp | 2 +- src/TableFunctions/TableFunctionRemote.cpp | 4 ++-- 11 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 89ff019ba6e..824a96fc1ae 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1317,7 +1317,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (insert && insert->select) insert->tryFindInputFunction(input_function); - bool is_async_insert = global_context->getSettings().async_insert && insert && insert->hasInlinedData(); + bool is_async_insert = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 9dbe611537b..8033d65c549 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -316,7 +316,7 @@ getTableOutput(const String & database_name, const String & table_name, ContextM return std::move(res.pipeline); } -static inline String reWriteMysqlQueryColumn(mysqlxx::Pool::Entry & connection, const String & database_name, const String & table_name, const Settings & global_settings) +static inline String rewriteMysqlQueryColumn(mysqlxx::Pool::Entry & connection, const String & database_name, const String & table_name, const Settings & global_settings) { Block tables_columns_sample_block { @@ -376,7 +376,7 @@ static inline void dumpDataForTables( auto pipeline = getTableOutput(database_name, table_name, query_context); StreamSettings mysql_input_stream_settings(context->getSettingsRef()); - String mysql_select_all_query = "SELECT " + reWriteMysqlQueryColumn(connection, mysql_database_name, table_name, context->getSettings()) + " FROM " + String mysql_select_all_query = "SELECT " + rewriteMysqlQueryColumn(connection, mysql_database_name, table_name, context->getSettingsRef()) + " FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name); LOG_INFO(&Poco::Logger::get("MaterializedMySQLSyncThread(" + database_name + ")"), "mysql_select_all_query is {}", mysql_select_all_query); auto input = std::make_unique(connection, mysql_select_all_query, pipeline.getHeader(), mysql_input_stream_settings); diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 48ddeed7fa6..62598c966e5 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -197,7 +197,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) size_t max_command_execution_time = config.getUInt64(settings_config_prefix + ".max_command_execution_time", 10); - size_t max_execution_time_seconds = static_cast(context->getSettings().max_execution_time.totalSeconds()); + size_t max_execution_time_seconds = static_cast(context->getSettingsRef().max_execution_time.totalSeconds()); if (max_execution_time_seconds != 0 && max_command_execution_time > max_execution_time_seconds) max_command_execution_time = max_execution_time_seconds; diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 8b68ba02504..df6e8ea98f5 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -229,7 +229,7 @@ ClusterPtr ClusterDiscovery::makeCluster(const ClusterInfo & cluster_info) bool secure = cluster_info.current_node.secure; auto cluster = std::make_shared( - context->getSettings(), + context->getSettingsRef(), shards, /* username= */ context->getUserName(), /* password= */ "", diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index b7e6f29188d..9a9a71f9688 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -921,8 +921,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( auto reader_settings = getMergeTreeReaderSettings(context); - bool use_skip_indexes = context->getSettings().use_skip_indexes; - if (select.final() && !context->getSettings().use_skip_indexes_if_final) + bool use_skip_indexes = settings.use_skip_indexes; + if (select.final() && !settings.use_skip_indexes_if_final) use_skip_indexes = false; result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 64c8a01bb9c..4ee3f2d4b82 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -59,7 +59,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery( : std::make_unique(); /// Create a source from input buffer using format from query - auto source = context->getInputFormat(ast_insert_query->format, *input_buffer, header, context->getSettings().max_insert_block_size); + auto source = context->getInputFormat(ast_insert_query->format, *input_buffer, header, context->getSettingsRef().max_insert_block_size); source->addBuffer(std::move(input_buffer)); return source; } diff --git a/src/Server/HTTP/HTTPServerConnection.cpp b/src/Server/HTTP/HTTPServerConnection.cpp index 7020b8e9a23..e365c9f31d0 100644 --- a/src/Server/HTTP/HTTPServerConnection.cpp +++ b/src/Server/HTTP/HTTPServerConnection.cpp @@ -36,7 +36,7 @@ void HTTPServerConnection::run() if (request.isSecure()) { - size_t hsts_max_age = context->getSettings().hsts_max_age.value; + size_t hsts_max_age = context->getSettingsRef().hsts_max_age.value; if (hsts_max_age > 0) response.add("Strict-Transport-Security", "max-age=" + std::to_string(hsts_max_age)); diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index dfe1ea6ffd3..7b370b7e63f 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -69,7 +69,7 @@ Pipe StorageHDFSCluster::read( size_t /*max_block_size*/, unsigned /*num_streams*/) { - auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettings()); + auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); auto iterator = std::make_shared(context, uri); auto callback = std::make_shared([iterator]() mutable -> String diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 762eb079c1c..57220c68347 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -82,7 +82,7 @@ Pipe StorageS3Cluster::read( { StorageS3::updateClientAndAuthSettings(context, client_auth); - auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettings()); + auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); StorageS3::updateClientAndAuthSettings(context, client_auth); auto iterator = std::make_shared(*client_auth.client, client_auth.uri); diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 6c8159ca720..f4dd9cbd45d 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -66,7 +66,7 @@ StoragesInfo::getParts(MergeTreeData::DataPartStateVector & state, bool has_stat } StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) - : query_id(context->getCurrentQueryId()), settings(context->getSettings()) + : query_id(context->getCurrentQueryId()), settings(context->getSettingsRef()) { /// Will apply WHERE to subset of columns and then add more columns. /// This is kind of complicated, but we use WHERE to do less work. diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 85857011616..90fbb079bb6 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -205,7 +205,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (name != "clusterAllReplicas") cluster = context->getCluster(cluster_name_expanded); else - cluster = context->getCluster(cluster_name_expanded)->getClusterWithReplicasAsShards(context->getSettings()); + cluster = context->getCluster(cluster_name_expanded)->getClusterWithReplicasAsShards(context->getSettingsRef()); } else { @@ -241,7 +241,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr bool treat_local_as_remote = false; bool treat_local_port_as_remote = context->getApplicationType() == Context::ApplicationType::LOCAL; cluster = std::make_shared( - context->getSettings(), + context->getSettingsRef(), names, configuration.username, configuration.password, From 8385dc27c2f43414a283a4faf07a08711a90274a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 10 Feb 2022 12:19:01 +0300 Subject: [PATCH 063/164] Fix test --- tests/queries/0_stateless/02210_append_to_dev_dull.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02210_append_to_dev_dull.sql b/tests/queries/0_stateless/02210_append_to_dev_dull.sql index 4ae77132972..a8aaa2f05ab 100644 --- a/tests/queries/0_stateless/02210_append_to_dev_dull.sql +++ b/tests/queries/0_stateless/02210_append_to_dev_dull.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + insert into table function file('/dev/null', 'Parquet', 'number UInt64') select * from numbers(10); insert into table function file('/dev/null', 'ORC', 'number UInt64') select * from numbers(10); insert into table function file('/dev/null', 'JSON', 'number UInt64') select * from numbers(10); From 87350f3552538bf797e42898fbf262cd1326f926 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 10 Feb 2022 17:33:12 +0800 Subject: [PATCH 064/164] Keeper SnapshotableHashTable fix clean bug. --- src/Coordination/SnapshotableHashTable.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index a837e51a9be..304183e02bb 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -296,19 +296,16 @@ public: arena.free(const_cast(itr->key.data), itr->key.size); list.clear(); updateDataSize(CLEAR, 0, 0, 0); - snapshot_invalid_iters.clear(); } void enableSnapshotMode(size_t up_to_size) { snapshot_mode = true; snapshot_up_to_size = up_to_size; - snapshot_invalid_iters.clear(); } void disableSnapshotMode() { - snapshot_mode = false; snapshot_up_to_size = 0; } From 4d73fb7fb7a05d2677516380c43e6f146b4de05f Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 10 Feb 2022 17:56:41 +0800 Subject: [PATCH 065/164] Fix ut in SnapshotableHashTable --- src/Coordination/SnapshotableHashTable.h | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 304183e02bb..002f1e4e365 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -36,7 +36,8 @@ private: /// Allows to avoid additional copies in updateValue function size_t snapshot_up_to_size = 0; ArenaWithFreeLists arena; - std::vector snapshot_invalid_iters{100000}; + /// Collect invalid iterators to avoid traversing the whole list + std::vector snapshot_invalid_iters; uint64_t approximate_data_size{0}; @@ -197,9 +198,9 @@ public: if (snapshot_mode) { list_itr->active_in_map = false; + snapshot_invalid_iters.push_back(list_itr); list_itr->free_key = true; map.erase(it->getKey()); - snapshot_invalid_iters.push_back(list_itr); } else { @@ -238,11 +239,11 @@ public: { auto elem_copy = *(list_itr); list_itr->active_in_map = false; + snapshot_invalid_iters.push_back(list_itr); updater(elem_copy.value); auto itr = list.insert(list.end(), elem_copy); it->getMapped() = itr; ret = itr; - snapshot_invalid_iters.push_back(list_itr); } else { From ff55f8d8d983f2efe1c4585365343949223872a9 Mon Sep 17 00:00:00 2001 From: cnmade Date: Thu, 10 Feb 2022 18:02:40 +0800 Subject: [PATCH 066/164] Translate zh/engines/database-engines/materialized-mysql: translate to zh and update index.md --- .../database-engines/materialized-mysql.md | 202 ++++++++---------- docs/zh/engines/database-engines/index.md | 2 +- 2 files changed, 94 insertions(+), 110 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index bcb026aa0dc..f654013494a 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -5,14 +5,15 @@ toc_title: MaterializedMySQL # [experimental] MaterializedMySQL {#materialized-mysql} -!!! warning "Warning" - This is an experimental feature that should not be used in production. +!!! warning "警告" + 这是一个实验性的特性,不应该在生产中使用. -Creates ClickHouse database with all the tables existing in MySQL, and all the data in those tables. -ClickHouse server works as MySQL replica. It reads binlog and performs DDL and DML queries. +创建ClickHouse数据库,包含MySQL中所有的表,以及这些表中的所有数据。 -## Creating a Database {#creating-a-database} +ClickHouse服务器作为MySQL副本工作。它读取binlog并执行DDL和DML查询。 + +## 创建数据库 {#creating-a-database} ``` sql CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] @@ -20,22 +21,23 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo [TABLE OVERRIDE table1 (...), TABLE OVERRIDE table2 (...)] ``` -**Engine Parameters** +**引擎参数** -- `host:port` — MySQL server endpoint. -- `database` — MySQL database name. -- `user` — MySQL user. -- `password` — User password. +- `host:port` — MySQL 服务地址. +- `database` — MySQL 数据库名称. +- `user` — MySQL 用户名. +- `password` — MySQL 用户密码. -**Engine Settings** +**引擎配置** -- `max_rows_in_buffer` — Maximum number of rows that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. -- `max_bytes_in_buffer` — Maximum number of bytes that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. -- `max_rows_in_buffers` — Maximum number of rows that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. -- `max_bytes_in_buffers` — Maximum number of bytes that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. -- `max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`. -- `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disables retry. Default: `1000`. -- `allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`). + +- `max_rows_in_buffer` — 允许在内存中缓存数据的最大行数(对于单个表和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值:`65 505`。 +- `max_bytes_in_buffer` - 允许在内存中缓存数据的最大字节数(对于单个表和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值: `1 048 576 `。 +- `max_rows_in_buffers` - 允许在内存中缓存数据的最大行数(用于数据库和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值: `65 505`。 +- `max_bytes_in_buffers` - 允许在内存中缓存数据的最大字节数(用于数据库和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值: `1 048 576`。 +- `max_flush_data_time ` - 允许数据在内存中缓存的最大毫秒数(对于数据库和无法查询的缓存数据)。当超过这个时间,数据将被物化。默认值: `1000`。 +- `max_wait_time_when_mysql_unavailable` - MySQL不可用时的重试间隔(毫秒)。负值禁用重试。默认值:`1000`。 +— `allows_query_when_mysql_lost `—允许在MySQL丢失时查询物化表。默认值:`0`(`false`)。 ```sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') @@ -44,26 +46,26 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', max_wait_time_when_mysql_unavailable=10000; ``` -**Settings on MySQL-server Side** +**MySQL服务器端配置** -For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: +为了`MaterializedMySQL`的正确工作,有一些必须设置的`MySQL`端配置设置: -- `default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method. -- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializedMySQL` replication. +- `default_authentication_plugin = mysql_native_password `,因为 `MaterializedMySQL` 只能授权使用该方法。 +- `gtid_mode = on`,因为基于GTID的日志记录是提供正确的 `MaterializedMySQL`复制的强制要求。 -!!! attention "Attention" - While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = on`. +!!! attention "注意" + 当打开`gtid_mode`时,您还应该指定`enforce_gtid_consistency = on`。 -## Virtual Columns {#virtual-columns} +## 虚拟列 {#virtual-columns} -When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. +当使用`MaterializeMySQL`数据库引擎时,[ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md)表与虚拟的`_sign`和`_version`列一起使用。 -- `_version` — Transaction counter. Type [UInt64](../../sql-reference/data-types/int-uint.md). -- `_sign` — Deletion mark. Type [Int8](../../sql-reference/data-types/int-uint.md). Possible values: - - `1` — Row is not deleted, - - `-1` — Row is deleted. +- `_version` — 事务版本. 类型 [UInt64](../../sql-reference/data-types/int-uint.md). +- `_sign` — 删除标记. 类型 [Int8](../../sql-reference/data-types/int-uint.md). 可能的值: + - `1` — 行没有删除, + - `-1` — 行已被删除. -## Data Types Support {#data_types-support} +## 支持的数据类型 {#data_types-support} | MySQL | ClickHouse | |-------------------------|--------------------------------------------------------------| @@ -89,91 +91,77 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree]( | BIT | [UInt64](../../sql-reference/data-types/int-uint.md) | | SET | [UInt64](../../sql-reference/data-types/int-uint.md) | -[Nullable](../../sql-reference/data-types/nullable.md) is supported. +[Nullable](../../sql-reference/data-types/nullable.md) 已经被支持. -The data of TIME type in MySQL is converted to microseconds in ClickHouse. +MySQL中的Time 类型,会被ClickHouse转换成微秒来存储 -Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication. +不支持其他类型。如果MySQL表包含此类类型的列,ClickHouse抛出异常"Unhandled data type"并停止复制。 -## Specifics and Recommendations {#specifics-and-recommendations} +## 规范和推荐用法 {#specifics-and-recommendations} -### Compatibility Restrictions {#compatibility-restrictions} +### 兼容性限制 {#compatibility-restrictions} -Apart of the data types limitations there are few restrictions comparing to `MySQL` databases, that should be resolved before replication will be possible: +除了数据类型的限制之外,还有一些限制与`MySQL`数据库相比有所不同,这应该在复制之前解决: -- Each table in `MySQL` should contain `PRIMARY KEY`. - -- Replication for tables, those are containing rows with `ENUM` field values out of range (specified in `ENUM` signature) will not work. +- `MySQL` 中的每个表都应该包含 `PRIMARY KEY`。 +- 对于表的复制,那些包含 `ENUM` 字段值超出范围的行(在 `ENUM` 签名中指定)将不起作用。 ### DDL Queries {#ddl-queries} -MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). If ClickHouse cannot parse some DDL query, the query is ignored. +MySQL DDL 语句会被转换成对应的ClickHouse DDL 语句,比如: ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). 如果ClickHouse 无法解析某些语句DDL 操作,则会跳过。 -### Data Replication {#data-replication} -`MaterializedMySQL` does not support direct `INSERT`, `DELETE` and `UPDATE` queries. However, they are supported in terms of data replication: +### 数据复制 {#data-replication} -- MySQL `INSERT` query is converted into `INSERT` with `_sign=1`. +MaterializedMySQL不支持直接的 `INSERT`, `DELETE` 和 `UPDATE` 查询。然而,它们在数据复制方面得到了支持: -- MySQL `DELETE` query is converted into `INSERT` with `_sign=-1`. +- MySQL `INSERT`查询被转换为`_sign=1`的INSERT查询。 +- MySQL `DELETE`查询被转换为`INSERT`,并且`_sign=-1`。 +- 如果主键被修改了,MySQL的 `UPDATE` 查询将被转换为 `INSERT` 带 `_sign=1` 和INSERT 带有_sign=-1;如果主键没有被修改,则转换为`INSERT`和`_sign=1`。 -- MySQL `UPDATE` query is converted into `INSERT` with `_sign=-1` and `INSERT` with `_sign=1` if the primary key has been changed, or - `INSERT` with `_sign=1` if not. +### MaterializedMySQL 数据表查询 {#select} -### Selecting from MaterializedMySQL Tables {#select} +`SELECT` 查询从 `MaterializedMySQL`表有一些细节: -`SELECT` query from `MaterializedMySQL` tables has some specifics: + - 如果在SELECT查询中没有指定`_version`,则 [FINAL](../../sql-reference/statements/select/from.md#select-from- FINAL)修饰符被使用,所以只有带有 `MAX(_version)`的行会返回每个主键值。 -- If `_version` is not specified in the `SELECT` query, the - [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier is used, so only rows with - `MAX(_version)` are returned for each primary key value. + - 如果在SELECT查询中没有指定 `_sign`,则默认使用 `WHERE _sign=1 `。所以被删除的行不是 +包含在结果集中。 -- If `_sign` is not specified in the `SELECT` query, `WHERE _sign=1` is used by default. So the deleted rows are not - included into the result set. + - 结果包括列注释,以防MySQL数据库表中存在这些列注释。 -- The result includes columns comments in case they exist in MySQL database tables. +### 索引转换 {#index-conversion} -### Index Conversion {#index-conversion} +在ClickHouse表中,MySQL的 `PRIMARY KEY` 和 `INDEX` 子句被转换为 `ORDER BY` 元组。 -MySQL `PRIMARY KEY` and `INDEX` clauses are converted into `ORDER BY` tuples in ClickHouse tables. +ClickHouse只有一个物理排序,由 `order by` 条件决定。要创建一个新的物理排序,请使用[materialized views](../../sql-reference/statements/create/view.md#materialized)。 -ClickHouse has only one physical order, which is determined by `ORDER BY` clause. To create a new physical order, use -[materialized views](../../sql-reference/statements/create/view.md#materialized). +**注意** -**Notes** +- `_sign=-1` 的行不会被物理地从表中删除。 +- 级联 `UPDATE/DELETE` 查询不支持 `MaterializedMySQL` 引擎,因为他们在 MySQL binlog中不可见的 +— 复制很容易被破坏。 +— 禁止对数据库和表进行手工操作。 +- `MaterializedMySQL` 受[optimize_on_insert](../../operations/settings/settings.md#optimize-on-insert)设置的影响。当MySQL服务器中的一个表发生变化时,数据会合并到 `MaterializedMySQL` 数据库中相应的表中。 -- Rows with `_sign=-1` are not deleted physically from the tables. -- Cascade `UPDATE/DELETE` queries are not supported by the `MaterializedMySQL` engine, as they are not visible in the - MySQL binlog. -- Replication can be easily broken. -- Manual operations on database and tables are forbidden. -- `MaterializedMySQL` is affected by the [optimize_on_insert](../../operations/settings/settings.md#optimize-on-insert) - setting. Data is merged in the corresponding table in the `MaterializedMySQL` database when a table in the MySQL - server changes. +### 表重写 {#table-overrides} -### Table Overrides {#table-overrides} +表覆盖可用于自定义ClickHouse DDL查询,从而允许您对应用程序进行模式优化。这对于控制分区特别有用,分区对MaterializedMySQL的整体性能非常重要。 -Table overrides can be used to customize the ClickHouse DDL queries, allowing you to make schema optimizations for your -application. This is especially useful for controlling partitioning, which is important for the overall performance of -MaterializedMySQL. +这些是你可以对MaterializedMySQL表重写的模式转换操作: -These are the schema conversion manipulations you can do with table overrides for MaterializedMySQL: - - * Modify column type. Must be compatible with the original type, or replication will fail. For example, - you can modify a UInt32 column to UInt64, but you can not modify a String column to Array(String). - * Modify [column TTL](../table-engines/mergetree-family/mergetree/#mergetree-column-ttl). - * Modify [column compression codec](../../sql-reference/statements/create/table/#codecs). - * Add [ALIAS columns](../../sql-reference/statements/create/table/#alias). - * Add [skipping indexes](../table-engines/mergetree-family/mergetree/#table_engine-mergetree-data_skipping-indexes) - * Add [projections](../table-engines/mergetree-family/mergetree/#projections). Note that projection optimizations are - disabled when using `SELECT ... FINAL` (which MaterializedMySQL does by default), so their utility is limited here. - `INDEX ... TYPE hypothesis` as [described in the v21.12 blog post]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/) - may be more useful in this case. - * Modify [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key/) - * Modify [ORDER BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * Modify [PRIMARY KEY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * Add [SAMPLE BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * Add [table TTL](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * 修改列类型。必须与原始类型兼容,否则复制将失败。例如,可以将`UInt32`列修改为`UInt64`,不能将 `String` 列修改为 `Array(String)`。 + * 修改 [column TTL](../table-engines/mergetree-family/mergetree/#mergetree-column-ttl). + * 修改 [column compression codec](../../sql-reference/statements/create/table/#codecs). + * 增加 [ALIAS columns](../../sql-reference/statements/create/table/#alias). + * 增加 [skipping indexes](../table-engines/mergetree-family/mergetree/#table_engine-mergetree-data_skipping-indexes) + * 增加 [projections](../table-engines/mergetree-family/mergetree/#projections). + 请注意,当使用 `SELECT ... FINAL ` (MaterializedMySQL默认是这样做的) 时,预测优化是被禁用的,所以这里是受限的, `INDEX ... TYPE hypothesis `[在v21.12的博客文章中描述]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/)可能在这种情况下更有用。 + * 修改 [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key/) + * 修改 [ORDER BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * 修改 [PRIMARY KEY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * 增加 [SAMPLE BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * 增加 [table TTL](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) ```sql CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) @@ -192,7 +180,7 @@ CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) ), ...] ``` -Example: +示例: ```sql CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) @@ -212,24 +200,20 @@ TABLE OVERRIDE table2 ( ) ``` -The `COLUMNS` list is sparse; existing columns are modified as specified, extra ALIAS columns are added. It is not -possible to add ordinary or MATERIALIZED columns. Modified columns with a different type must be assignable from the -original type. There is currently no validation of this or similar issues when the `CREATE DATABASE` query executes, so -extra care needs to be taken. -You may specify overrides for tables that do not exist yet. +`COLUMNS`列表是稀疏的;根据指定修改现有列,添加额外的ALIAS列。不可能添加普通列或实体化列。具有不同类型的已修改列必须可从原始类型赋值。在执行`CREATE DATABASE` 查询时,目前还没有验证这个或类似的问题,因此需要格外小心。 -!!! warning "Warning" - It is easy to break replication with table overrides if not used with care. For example: - - * If an ALIAS column is added with a table override, and a column with the same name is later added to the source - MySQL table, the converted ALTER TABLE query in ClickHouse will fail and replication stops. - * It is currently possible to add overrides that reference nullable columns where not-nullable are required, such as in - `ORDER BY` or `PARTITION BY`. This will cause CREATE TABLE queries that will fail, also causing replication to stop. +您可以为还不存在的表指定重写。 -## Examples of Use {#examples-of-use} +!!! warning "警告" + 如果使用时不小心,很容易用表重写中断复制。例如: -Queries in MySQL: + * 如果一个ALIAS列被添加了一个表覆盖,并且一个具有相同名称的列后来被添加到源MySQL表,在ClickHouse中转换后的ALTER table查询将失败并停止复制。 + * 目前可以添加引用可空列的覆盖,而非空列是必需的,例如 `ORDER BY` 或 `PARTITION BY`。这将导致CREATE TABLE查询失败,也会导致复制停止。 + +## 使用示例 {#examples-of-use} + + MySQL 查询语句: ``` sql mysql> CREATE DATABASE db; @@ -247,9 +231,9 @@ mysql> SELECT * FROM test; └───┴─────┴──────┘ ``` -Database in ClickHouse, exchanging data with the MySQL server: +ClickHouse中的数据库,与MySQL服务器交换数据: -The database and the table created: +创建的数据库和表: ``` sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***'); @@ -262,7 +246,7 @@ SHOW TABLES FROM mysql; └──────┘ ``` -After inserting data: +数据插入之后: ``` sql SELECT * FROM mysql.test; @@ -275,7 +259,7 @@ SELECT * FROM mysql.test; └───┴────┘ ``` -After deleting data, adding the column and updating: +删除数据后,添加列并更新: ``` sql SELECT * FROM mysql.test; @@ -287,4 +271,4 @@ SELECT * FROM mysql.test; └───┴─────┴──────┘ ``` -[Original article](https://clickhouse.com/docs/en/engines/database-engines/materialized-mysql/) +[来源文章](https://clickhouse.com/docs/en/engines/database-engines/materialized-mysql/) diff --git a/docs/zh/engines/database-engines/index.md b/docs/zh/engines/database-engines/index.md index e4647da154d..0d844365fbb 100644 --- a/docs/zh/engines/database-engines/index.md +++ b/docs/zh/engines/database-engines/index.md @@ -14,7 +14,7 @@ toc_title: Introduction - [MySQL](../../engines/database-engines/mysql.md) -- [MaterializeMySQL](../../engines/database-engines/materialize-mysql.md) +- [MaterializeMySQL](../../engines/database-engines/materialized-mysql.md) - [Lazy](../../engines/database-engines/lazy.md) From 3e21ebeb02ee8bb1e2815ef01fd47f819a2358b6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 8 Feb 2022 09:49:15 +0000 Subject: [PATCH 067/164] For SQLUserDefinedFunctions change access type from DATABASE to GLOBAL --- src/Access/Common/AccessType.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 55b8359d385..16d3de40ec3 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -86,7 +86,7 @@ enum class AccessType M(CREATE_DICTIONARY, "", DICTIONARY, CREATE) /* allows to execute {CREATE|ATTACH} DICTIONARY */\ M(CREATE_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables; implicitly enabled by the grant CREATE_TABLE on any table */ \ - M(CREATE_FUNCTION, "", DATABASE, CREATE) /* allows to execute CREATE FUNCTION */ \ + M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\ @@ -94,7 +94,7 @@ enum class AccessType M(DROP_VIEW, "", VIEW, DROP) /* allows to execute {DROP|DETACH} TABLE for views; implicitly enabled by the grant DROP_TABLE */\ M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\ - M(DROP_FUNCTION, "", DATABASE, DROP) /* allows to execute DROP FUNCTION */\ + M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \ From f7b70877c8a07a53774019ec1cb0f033b071ec75 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 9 Feb 2022 11:57:11 +0000 Subject: [PATCH 068/164] Fixed tests --- tests/queries/0_stateless/01271_show_privileges.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index c18992583cd..b2b02b24cb0 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -49,13 +49,13 @@ CREATE TABLE [] TABLE CREATE CREATE VIEW [] VIEW CREATE CREATE DICTIONARY [] DICTIONARY CREATE CREATE TEMPORARY TABLE [] GLOBAL CREATE -CREATE FUNCTION [] DATABASE CREATE +CREATE FUNCTION [] GLOBAL CREATE CREATE [] \N ALL DROP DATABASE [] DATABASE DROP DROP TABLE [] TABLE DROP DROP VIEW [] VIEW DROP DROP DICTIONARY [] DICTIONARY DROP -DROP FUNCTION [] DATABASE DROP +DROP FUNCTION [] GLOBAL DROP DROP [] \N ALL TRUNCATE ['TRUNCATE TABLE'] TABLE ALL OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL From 9e58ae7577e0cb378c3c02c40c2b86c812cf0e06 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Feb 2022 16:00:37 +0300 Subject: [PATCH 069/164] Support jsonl extension for JSONEachRow format --- .../Impl/JSONEachRowRowInputFormat.cpp | 1 + .../02211_jsonl_format_extension.reference | 20 +++++++++++++++++++ .../02211_jsonl_format_extension.sql | 2 ++ 3 files changed, 23 insertions(+) create mode 100644 tests/queries/0_stateless/02211_jsonl_format_extension.reference create mode 100644 tests/queries/0_stateless/02211_jsonl_format_extension.sql diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 6d546a3b772..549fd7a6113 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -356,6 +356,7 @@ void registerInputFormatJSONEachRow(FormatFactory & factory) }); factory.registerFileExtension("ndjson", "JSONEachRow"); + factory.registerFileExtension("jsonl", "JSONEachRow"); factory.registerInputFormat("JSONStringsEachRow", []( ReadBuffer & buf, diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.reference b/tests/queries/0_stateless/02211_jsonl_format_extension.reference new file mode 100644 index 00000000000..7b36cc96f5e --- /dev/null +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.reference @@ -0,0 +1,20 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.sql b/tests/queries/0_stateless/02211_jsonl_format_extension.sql new file mode 100644 index 00000000000..889db34973c --- /dev/null +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.sql @@ -0,0 +1,2 @@ +insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10); +select * from file('data.jsonl'); From d40552359868d09aec65393528dc30ed5f5df139 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 10 Feb 2022 13:27:27 +0000 Subject: [PATCH 070/164] Fix ProfileEvents sending in INSERT SELECT --- src/Server/TCPHandler.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 668017f8ef8..99523ff09e3 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -359,6 +359,7 @@ void TCPHandler::runImpl() return true; sendProgress(); + sendProfileEvents(); sendLogs(); return false; From f8c8eeceead2a8df29b1be703c6a558c10c8814f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 10 Feb 2022 20:43:12 +0700 Subject: [PATCH 071/164] Add field output_format to first Result sent in gRPC protocol. --- src/Server/GRPCServer.cpp | 8 ++++++++ src/Server/grpc_protos/clickhouse_grpc.proto | 6 +++++- tests/integration/test_grpc_protocol/test.py | 14 ++++++++++++++ 3 files changed, 27 insertions(+), 1 deletion(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 10bbce24913..320cdc78055 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -642,6 +642,7 @@ namespace void throwIfFailedToReadQueryInfo(); bool isQueryCancelled(); + void addOutputFormatToResult(); void addProgressToResult(); void addTotalsToResult(const Block & totals); void addExtremesToResult(const Block & extremes); @@ -1189,6 +1190,8 @@ namespace return true; }; + addOutputFormatToResult(); + Block block; while (check_for_cancel()) { @@ -1439,6 +1442,11 @@ namespace return false; } + void Call::addOutputFormatToResult() + { + *result.mutable_output_format() = output_format; + } + void Call::addProgressToResult() { auto values = progress.fetchAndResetPiecewiseAtomically(); diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index f596c3b7d6d..3770c1a7265 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -187,7 +187,11 @@ message Exception { // Result of execution of a query which is sent back by the ClickHouse server to the client. message Result { - // Output of the query, represented in the `output_format` or in a format specified in `query`. + // The format in which `output`, `totals` and `extremes` are written. + // It's either the same as `output_format` specified in `QueryInfo` or the format specified in the query itself. + string output_format = 9; + + // Output of the query, represented in the `output_format`. bytes output = 1; bytes totals = 2; bytes extremes = 3; diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 65ee3cb4261..521bdcb1843 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -204,6 +204,19 @@ def test_totals_and_extremes(): assert query("SELECT x, y FROM t") == "1\t2\n2\t4\n3\t2\n3\t3\n3\t4\n" assert query_and_get_extremes("SELECT x, y FROM t", settings={"extremes": "1"}) == "1\t2\n3\t4\n" +def test_get_query_details(): + result = list(query_no_errors("CREATE TABLE t (a UInt8) ENGINE = Memory"))[0] + assert result.output_format == '' + assert result.output == b'' + # + result = list(query_no_errors("SELECT 'a', 1", query_id = '', output_format = 'TabSeparated'))[0] + assert result.output_format == 'TabSeparated' + assert result.output == b'a\t1\n' + # + result = list(query_no_errors("SELECT 'a' AS x, 1 FORMAT JSONEachRow"))[0] + assert result.output_format == 'JSONEachRow' + assert result.output == b'{"x":"a","1":1}\n' + def test_errors_handling(): e = query_and_get_error("") #print(e) @@ -232,6 +245,7 @@ def test_progress(): read_bytes: 16 total_rows_to_read: 8 } +output_format: "TabSeparated" , output: "0\\t0\\n1\\t0\\n" , progress { read_rows: 2 From bd746fd82b4bbded579073a4fe70f08b0b10bf9c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 8 Feb 2022 20:15:56 +0700 Subject: [PATCH 072/164] Add fields query_id and time_zone to first Result sent in gRPC protocol. --- src/Server/GRPCServer.cpp | 10 ++++++++++ src/Server/grpc_protos/clickhouse_grpc.proto | 5 ++++- tests/integration/test_grpc_protocol/test.py | 15 +++++++++++++-- 3 files changed, 27 insertions(+), 3 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 320cdc78055..4aed9204f13 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -642,6 +642,7 @@ namespace void throwIfFailedToReadQueryInfo(); bool isQueryCancelled(); + void addQueryDetailsToResult(); void addOutputFormatToResult(); void addProgressToResult(); void addTotalsToResult(const Block & totals); @@ -1151,6 +1152,9 @@ namespace void Call::generateOutput() { + /// We add query_id and time_zone to the first result anyway. + addQueryDetailsToResult(); + if (!io.pipeline.initialized() || io.pipeline.pushing()) return; @@ -1442,6 +1446,12 @@ namespace return false; } + void Call::addQueryDetailsToResult() + { + *result.mutable_query_id() = query_context->getClientInfo().current_query_id; + *result.mutable_time_zone() = DateLUT::instance().getTimeZone(); + } + void Call::addOutputFormatToResult() { *result.mutable_output_format() = output_format; diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index 3770c1a7265..d72ef5c423a 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -187,9 +187,12 @@ message Exception { // Result of execution of a query which is sent back by the ClickHouse server to the client. message Result { + string query_id = 9; + string time_zone = 10; + // The format in which `output`, `totals` and `extremes` are written. // It's either the same as `output_format` specified in `QueryInfo` or the format specified in the query itself. - string output_format = 9; + string output_format = 11; // Output of the query, represented in the `output_format`. bytes output = 1; diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 521bdcb1843..4edbadef8f0 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -2,6 +2,8 @@ import os import pytest import sys import time +import pytz +import uuid import grpc from helpers.cluster import ClickHouseCluster, run_and_check from threading import Thread @@ -205,15 +207,21 @@ def test_totals_and_extremes(): assert query_and_get_extremes("SELECT x, y FROM t", settings={"extremes": "1"}) == "1\t2\n3\t4\n" def test_get_query_details(): - result = list(query_no_errors("CREATE TABLE t (a UInt8) ENGINE = Memory"))[0] + result = list(query_no_errors("CREATE TABLE t (a UInt8) ENGINE = Memory", query_id = '123'))[0] + assert result.query_id == '123' + pytz.timezone(result.time_zone) assert result.output_format == '' assert result.output == b'' # result = list(query_no_errors("SELECT 'a', 1", query_id = '', output_format = 'TabSeparated'))[0] + uuid.UUID(result.query_id) + pytz.timezone(result.time_zone) assert result.output_format == 'TabSeparated' assert result.output == b'a\t1\n' # - result = list(query_no_errors("SELECT 'a' AS x, 1 FORMAT JSONEachRow"))[0] + result = list(query_no_errors("SELECT 'a' AS x, 1 FORMAT JSONEachRow", query_id = ''))[0] + uuid.UUID(result.query_id) + pytz.timezone(result.time_zone) assert result.output_format == 'JSONEachRow' assert result.output == b'{"x":"a","1":1}\n' @@ -238,6 +246,9 @@ def test_logs(): def test_progress(): results = query_no_errors("SELECT number, sleep(0.31) FROM numbers(8) SETTINGS max_block_size=2, interactive_delay=100000", stream_output=True) + for result in results: + result.time_zone = '' + result.query_id = '' #print(results) assert str(results) ==\ """[progress { From cf2e205b08b99285afa96a12cf8b4ec273952408 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 8 Feb 2022 20:15:56 +0700 Subject: [PATCH 073/164] Add field output_columns to first Result sent in gRPC protocol. --- src/Server/GRPCServer.cpp | 17 +++++++++++++++++ src/Server/grpc_protos/clickhouse_grpc.proto | 6 ++++++ tests/integration/test_grpc_protocol/test.py | 12 ++++++++---- 3 files changed, 31 insertions(+), 4 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 4aed9204f13..eeaf5b32a92 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -644,6 +644,7 @@ namespace void addQueryDetailsToResult(); void addOutputFormatToResult(); + void addOutputColumnsNamesAndTypesToResult(const Block & headers); void addProgressToResult(); void addTotalsToResult(const Block & totals); void addExtremesToResult(const Block & extremes); @@ -669,6 +670,7 @@ namespace CompressionMethod input_compression_method = CompressionMethod::None; PODArray output; String output_format; + bool send_output_columns_names_and_types = false; CompressionMethod output_compression_method = CompressionMethod::None; int output_compression_level = 0; @@ -890,6 +892,8 @@ namespace if (output_format.empty()) output_format = query_context->getDefaultFormat(); + send_output_columns_names_and_types = query_info.send_output_columns(); + /// Choose compression. String input_compression_method_str = query_info.input_compression_type(); if (input_compression_method_str.empty()) @@ -1195,6 +1199,7 @@ namespace }; addOutputFormatToResult(); + addOutputColumnsNamesAndTypesToResult(header); Block block; while (check_for_cancel()) @@ -1457,6 +1462,18 @@ namespace *result.mutable_output_format() = output_format; } + void Call::addOutputColumnsNamesAndTypesToResult(const Block & header) + { + if (!send_output_columns_names_and_types) + return; + for (const auto & column : header) + { + auto & name_and_type = *result.add_output_columns(); + *name_and_type.mutable_name() = column.name; + *name_and_type.mutable_type() = column.type->getName(); + } + } + void Call::addProgressToResult() { auto values = progress.fetchAndResetPiecewiseAtomically(); diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index d72ef5c423a..4593cfff096 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -82,6 +82,9 @@ message QueryInfo { // Default output format. If not specified, 'TabSeparated' is used. string output_format = 7; + // Set it if you want the names and the types of output columns to be sent to the client. + bool send_output_columns = 24; + repeated ExternalTable external_tables = 8; string user_name = 9; @@ -194,6 +197,9 @@ message Result { // It's either the same as `output_format` specified in `QueryInfo` or the format specified in the query itself. string output_format = 11; + // The names and types of columns of the result written in `output`. + repeated NameAndType output_columns = 12; + // Output of the query, represented in the `output_format`. bytes output = 1; bytes totals = 2; diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 4edbadef8f0..bd9a0cbe438 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -45,8 +45,8 @@ def create_channel(): main_channel = channel return channel -def query_common(query_text, settings={}, input_data=[], input_data_delimiter='', output_format='TabSeparated', external_tables=[], - user_name='', password='', query_id='123', session_id='', stream_output=False, channel=None): +def query_common(query_text, settings={}, input_data=[], input_data_delimiter='', output_format='TabSeparated', send_output_columns=False, + external_tables=[], user_name='', password='', query_id='123', session_id='', stream_output=False, channel=None): if type(input_data) is not list: input_data = [input_data] if type(input_data_delimiter) is str: @@ -60,7 +60,8 @@ def query_common(query_text, settings={}, input_data=[], input_data_delimiter='' input_data_part = input_data_part.encode(DEFAULT_ENCODING) return clickhouse_grpc_pb2.QueryInfo(query=query_text, settings=settings, input_data=input_data_part, input_data_delimiter=input_data_delimiter, output_format=output_format, - external_tables=external_tables, user_name=user_name, password=password, query_id=query_id, + send_output_columns=send_output_columns, external_tables=external_tables, + user_name=user_name, password=password, query_id=query_id, session_id=session_id, next_query_info=bool(input_data)) def send_query_info(): yield query_info() @@ -211,18 +212,21 @@ def test_get_query_details(): assert result.query_id == '123' pytz.timezone(result.time_zone) assert result.output_format == '' + assert len(result.output_columns) == 0 assert result.output == b'' # result = list(query_no_errors("SELECT 'a', 1", query_id = '', output_format = 'TabSeparated'))[0] uuid.UUID(result.query_id) pytz.timezone(result.time_zone) assert result.output_format == 'TabSeparated' + assert len(result.output_columns) == 0 assert result.output == b'a\t1\n' # - result = list(query_no_errors("SELECT 'a' AS x, 1 FORMAT JSONEachRow", query_id = ''))[0] + result = list(query_no_errors("SELECT 'a' AS x, 1 FORMAT JSONEachRow", query_id = '', send_output_columns=True))[0] uuid.UUID(result.query_id) pytz.timezone(result.time_zone) assert result.output_format == 'JSONEachRow' + assert ([(col.name, col.type) for col in result.output_columns]) == [('x', 'String'), ('1', 'UInt8')] assert result.output == b'{"x":"a","1":1}\n' def test_errors_handling(): From 80e110081303d9917e738f2c69eeb0b1bb451f63 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 10 Feb 2022 10:45:27 -0400 Subject: [PATCH 074/164] Virtual Columns MergeTree engine --- .../engines/table-engines/mergetree-family/mergetree.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 6769f48a466..92865c94475 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -886,3 +886,12 @@ S3 disk can be configured as `main` or `cold` storage: ``` In case of `cold` option a data can be moved to S3 if local disk free size will be smaller than `move_factor * disk_size` or by TTL move rule. + +## Virtual Columns {#virtual-columns} + +- `_part` — Name of a part. +- `_part_index` — Sequential index of the part in the query result. +- `_partition_id` — Name of a partition. +- `_part_uuid` — Unique part identifier (if enabled MergeTree setting `assign_part_uuids`). +- `_partition_value` — Values (a tuple) of a `partition by` expression. +- `_sample_factor` — Sample factor (from the query). From 13cf8f87e108f4ed7228370152c4147ac7117b9f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 10 Feb 2022 18:14:46 +0300 Subject: [PATCH 075/164] Fix test --- tests/queries/0_stateless/02211_jsonl_format_extension.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.sql b/tests/queries/0_stateless/02211_jsonl_format_extension.sql index 889db34973c..08fff5a11f5 100644 --- a/tests/queries/0_stateless/02211_jsonl_format_extension.sql +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.sql @@ -1,2 +1,3 @@ +-- Tags: no-fasttest insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10); select * from file('data.jsonl'); From 9bbbbdcdd67f3a7827dffa42fedc4e2eb21d60ef Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Feb 2022 15:24:13 +0300 Subject: [PATCH 076/164] Improve schema inference in clickhouse-local --- programs/local/LocalServer.cpp | 10 +++++++++- .../02211_shcema_inference_from_stdin.reference | 15 +++++++++++++++ .../02211_shcema_inference_from_stdin.sh | 17 +++++++++++++++++ 3 files changed, 41 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference create mode 100755 tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 70363c62cac..82070701e1c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -313,9 +313,17 @@ void LocalServer::cleanup() } +static bool checkIfStdinIsNotEmpty() +{ + struct stat file_stat; + return fstat(STDIN_FILENO, &file_stat) == 0 && (S_ISREG(file_stat.st_mode) || S_ISFIFO(file_stat.st_mode)); +} + std::string LocalServer::getInitialCreateTableQuery() { - if (!config().has("table-structure") && !config().has("table-file") && !config().has("table-data-format")) + /// We can create initial table only when we have data for it + /// in file or in stdin (we treat stdin as table data if we have query) + if (!config().has("table-file") && (!checkIfStdinIsNotEmpty() || !config().has("query"))) return {}; auto table_name = backQuoteIfNeed(config().getString("table-name", "table")); diff --git a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference new file mode 100644 index 00000000000..d176e0ee1ed --- /dev/null +++ b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference @@ -0,0 +1,15 @@ +x Nullable(Float64) +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +c1 Nullable(String) +c2 Nullable(String) +c3 Nullable(String) +1 2 3 diff --git a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh new file mode 100755 index 00000000000..8f767447e2d --- /dev/null +++ b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_LOCAL -q "select toUInt32(number) as x from numbers(10) format JSONEachRow" > data.jsoneachrow + +$CLICKHOUSE_LOCAL -q "desc table table" < data.jsoneachrow +$CLICKHOUSE_LOCAL -q "select * from table" < data.jsoneachrow + +rm data.jsoneachrow + +echo -e "1\t2\t3" | $CLICKHOUSE_LOCAL -q "desc table table" +echo -e "1\t2\t3" | $CLICKHOUSE_LOCAL -q "select * from table" + From a69905d378968866bb26b8f84f90179e15325dfe Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Feb 2022 16:57:32 +0300 Subject: [PATCH 077/164] Fix --- programs/local/LocalServer.cpp | 6 +++--- .../0_stateless/02211_shcema_inference_from_stdin.sh | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 82070701e1c..a0945bdeb67 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -313,17 +313,17 @@ void LocalServer::cleanup() } -static bool checkIfStdinIsNotEmpty() +static bool checkIfStdinIsRegularFile() { struct stat file_stat; - return fstat(STDIN_FILENO, &file_stat) == 0 && (S_ISREG(file_stat.st_mode) || S_ISFIFO(file_stat.st_mode)); + return fstat(STDIN_FILENO, &file_stat) == 0 && S_ISREG(file_stat.st_mode); } std::string LocalServer::getInitialCreateTableQuery() { /// We can create initial table only when we have data for it /// in file or in stdin (we treat stdin as table data if we have query) - if (!config().has("table-file") && (!checkIfStdinIsNotEmpty() || !config().has("query"))) + if (!config().has("table-file") && (!checkIfStdinIsRegularFile() || !config().has("query"))) return {}; auto table_name = backQuoteIfNeed(config().getString("table-name", "table")); diff --git a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh index 8f767447e2d..cb1753ad637 100755 --- a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh +++ b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh @@ -12,6 +12,6 @@ $CLICKHOUSE_LOCAL -q "select * from table" < data.jsoneachrow rm data.jsoneachrow -echo -e "1\t2\t3" | $CLICKHOUSE_LOCAL -q "desc table table" -echo -e "1\t2\t3" | $CLICKHOUSE_LOCAL -q "select * from table" +echo -e "1\t2\t3" | $CLICKHOUSE_LOCAL -q "desc table table" --file=- +echo -e "1\t2\t3" | $CLICKHOUSE_LOCAL -q "select * from table" --file=- From 7d92d9d06015a50cd3dfd020850c05be063c7d29 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Feb 2022 18:37:27 +0300 Subject: [PATCH 078/164] Fix tests --- programs/local/LocalServer.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index a0945bdeb67..f673aa9b5af 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -321,9 +321,7 @@ static bool checkIfStdinIsRegularFile() std::string LocalServer::getInitialCreateTableQuery() { - /// We can create initial table only when we have data for it - /// in file or in stdin (we treat stdin as table data if we have query) - if (!config().has("table-file") && (!checkIfStdinIsRegularFile() || !config().has("query"))) + if (!config().has("table-structure") && !config().has("table-file") && !config().has("table-data-format") && (!checkIfStdinIsRegularFile() || !config().has("query"))) return {}; auto table_name = backQuoteIfNeed(config().getString("table-name", "table")); From a1759d1aa0145b3941c8340f7ef2572246e17039 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Feb 2022 18:38:28 +0300 Subject: [PATCH 079/164] Fix test --- tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh index cb1753ad637..9b98e970729 100755 --- a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh +++ b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash - +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 70986a70a1ae8c8f0498c2bbec6b7cd4be982972 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 10 Feb 2022 19:18:01 +0300 Subject: [PATCH 080/164] support TTL TO [DISK|VOLUME] [IF EXISTS] --- src/Disks/IStoragePolicy.cpp | 34 ++++++++++++ src/Disks/IStoragePolicy.h | 6 ++- src/Disks/StoragePolicy.cpp | 6 +-- src/Disks/StoragePolicy.h | 4 +- src/Parsers/ASTTTLElement.cpp | 28 +++++++--- src/Parsers/ASTTTLElement.h | 4 +- src/Parsers/ExpressionElementParsers.cpp | 7 ++- src/Storages/MergeTree/MergeTreeData.cpp | 19 ++++--- src/Storages/TTLDescription.cpp | 5 +- src/Storages/TTLDescription.h | 4 ++ .../config.d/storage_configuration.xml | 8 +++ tests/integration/test_ttl_move/test.py | 54 +++++++++++++++++++ .../02207_ttl_move_if_exists.reference | 1 + .../0_stateless/02207_ttl_move_if_exists.sql | 9 ++++ 14 files changed, 161 insertions(+), 28 deletions(-) create mode 100644 src/Disks/IStoragePolicy.cpp create mode 100644 tests/queries/0_stateless/02207_ttl_move_if_exists.reference create mode 100644 tests/queries/0_stateless/02207_ttl_move_if_exists.sql diff --git a/src/Disks/IStoragePolicy.cpp b/src/Disks/IStoragePolicy.cpp new file mode 100644 index 00000000000..2ba6df4be8f --- /dev/null +++ b/src/Disks/IStoragePolicy.cpp @@ -0,0 +1,34 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_VOLUME; + extern const int UNKNOWN_DISK; +} + +DiskPtr IStoragePolicy::getDiskByName(const String & disk_name) const +{ + auto disk = tryGetDiskByName(disk_name); + if (!disk) + throw Exception(ErrorCodes::UNKNOWN_DISK, + "No such disk {} in storage policy {}", backQuote(disk_name), backQuote(getName())); + + return disk; +} + +VolumePtr IStoragePolicy::getVolumeByName(const String & volume_name) const +{ + auto volume = tryGetVolumeByName(volume_name); + if (!volume) + throw Exception(ErrorCodes::UNKNOWN_VOLUME, + "No such volume {} in storage policy {}", backQuote(volume_name), backQuote(getName())); + + return volume; +} + +} diff --git a/src/Disks/IStoragePolicy.h b/src/Disks/IStoragePolicy.h index 8c6d56bdcfd..a5d1120f377 100644 --- a/src/Disks/IStoragePolicy.h +++ b/src/Disks/IStoragePolicy.h @@ -39,7 +39,8 @@ public: /// Used when it's not important, for example for /// mutations files virtual DiskPtr getAnyDisk() const = 0; - virtual DiskPtr getDiskByName(const String & disk_name) const = 0; + virtual DiskPtr tryGetDiskByName(const String & disk_name) const = 0; + DiskPtr getDiskByName(const String & disk_name) const; /// Get free space from most free disk virtual UInt64 getMaxUnreservedFreeSpace() const = 0; /// Reserves space on any volume with index > min_volume_index or returns nullptr @@ -53,7 +54,8 @@ public: virtual ReservationPtr makeEmptyReservationOnLargestDisk() const = 0; /// Get volume by index. virtual VolumePtr getVolume(size_t index) const = 0; - virtual VolumePtr getVolumeByName(const String & volume_name) const = 0; + virtual VolumePtr tryGetVolumeByName(const String & volume_name) const = 0; + VolumePtr getVolumeByName(const String & volume_name) const; /// Checks if storage policy can be replaced by another one. virtual void checkCompatibleWith(const StoragePolicyPtr & new_storage_policy) const = 0; /// Find volume index, which contains disk diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 4c77528f1fc..20192c3a29f 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -179,7 +179,7 @@ DiskPtr StoragePolicy::getAnyDisk() const } -DiskPtr StoragePolicy::getDiskByName(const String & disk_name) const +DiskPtr StoragePolicy::tryGetDiskByName(const String & disk_name) const { for (auto && volume : volumes) for (auto && disk : volume->getDisks()) @@ -265,11 +265,11 @@ VolumePtr StoragePolicy::getVolume(size_t index) const } -VolumePtr StoragePolicy::getVolumeByName(const String & volume_name) const +VolumePtr StoragePolicy::tryGetVolumeByName(const String & volume_name) const { auto it = volume_index_by_volume_name.find(volume_name); if (it == volume_index_by_volume_name.end()) - throw Exception("No such volume " + backQuote(volume_name) + " in storage policy " + backQuote(name), ErrorCodes::UNKNOWN_VOLUME); + return nullptr; return getVolume(it->second); } diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index b09baf09bda..c3f72e01ec8 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -52,7 +52,7 @@ public: /// mutations files DiskPtr getAnyDisk() const override; - DiskPtr getDiskByName(const String & disk_name) const override; + DiskPtr tryGetDiskByName(const String & disk_name) const override; /// Get free space from most free disk UInt64 getMaxUnreservedFreeSpace() const override; @@ -84,7 +84,7 @@ public: /// Get volume by index. VolumePtr getVolume(size_t index) const override; - VolumePtr getVolumeByName(const String & volume_name) const override; + VolumePtr tryGetVolumeByName(const String & volume_name) const override; /// Checks if storage policy can be replaced by another one. void checkCompatibleWith(const StoragePolicyPtr & new_storage_policy) const override; diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 2d22c1b4307..90278e27c0c 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -1,13 +1,17 @@ - #include #include #include #include - +#include namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + ASTPtr ASTTTLElement::clone() const { auto clone = std::make_shared(*this); @@ -29,13 +33,21 @@ ASTPtr ASTTTLElement::clone() const void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { ttl()->formatImpl(settings, state, frame); - if (mode == TTLMode::MOVE && destination_type == DataDestinationType::DISK) + if (mode == TTLMode::MOVE) { - settings.ostr << " TO DISK " << quoteString(destination_name); - } - else if (mode == TTLMode::MOVE && destination_type == DataDestinationType::VOLUME) - { - settings.ostr << " TO VOLUME " << quoteString(destination_name); + if (destination_type == DataDestinationType::DISK) + settings.ostr << " TO DISK "; + else if (destination_type == DataDestinationType::VOLUME) + settings.ostr << " TO VOLUME "; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unsupported destination type {} for TTL MOVE", + magic_enum::enum_name(destination_type)); + + if (if_exists) + settings.ostr << "IF EXISTS "; + + settings.ostr << quoteString(destination_name); } else if (mode == TTLMode::GROUP_BY) { diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index ce011d76c7b..a396a4c54e0 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -16,16 +16,18 @@ public: TTLMode mode; DataDestinationType destination_type; String destination_name; + bool if_exists = false; ASTs group_by_key; ASTs group_by_assignments; ASTPtr recompression_codec; - ASTTTLElement(TTLMode mode_, DataDestinationType destination_type_, const String & destination_name_) + ASTTTLElement(TTLMode mode_, DataDestinationType destination_type_, const String & destination_name_, bool if_exists_) : mode(mode_) , destination_type(destination_type_) , destination_name(destination_name_) + , if_exists(if_exists_) , ttl_expr_pos(-1) , where_expr_pos(-1) { diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 9c8f8b4e46b..e00e0aba7b3 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2360,6 +2360,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_to_disk("TO DISK"); ParserKeyword s_to_volume("TO VOLUME"); + ParserKeyword s_if_exists("IF EXISTS"); ParserKeyword s_delete("DELETE"); ParserKeyword s_where("WHERE"); ParserKeyword s_group_by("GROUP BY"); @@ -2414,9 +2415,13 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr group_by_key; ASTPtr recompression_codec; ASTPtr group_by_assignments; + bool if_exists = false; if (mode == TTLMode::MOVE) { + if (s_if_exists.ignore(pos)) + if_exists = true; + ASTPtr ast_space_name; if (!parser_string_literal.parse(pos, ast_space_name, expected)) return false; @@ -2448,7 +2453,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - auto ttl_element = std::make_shared(mode, destination_type, destination_name); + auto ttl_element = std::make_shared(mode, destination_type, destination_name, if_exists); ttl_element->setTTL(std::move(ttl_expr)); if (where_expr) ttl_element->setWhere(std::move(where_expr)); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 68fa81e1df9..7ce98b36ed2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -650,13 +650,14 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta { for (const auto & move_ttl : new_table_ttl.move_ttl) { - if (!getDestinationForMoveTTL(move_ttl)) + if (!move_ttl.if_exists && !getDestinationForMoveTTL(move_ttl)) { String message; if (move_ttl.destination_type == DataDestinationType::DISK) - message = "No such disk " + backQuote(move_ttl.destination_name) + " for given storage policy."; + message = "No such disk " + backQuote(move_ttl.destination_name) + " for given storage policy"; else - message = "No such volume " + backQuote(move_ttl.destination_name) + " for given storage policy."; + message = "No such volume " + backQuote(move_ttl.destination_name) + " for given storage policy"; + throw Exception(message, ErrorCodes::BAD_TTL_EXPRESSION); } } @@ -3363,9 +3364,6 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); auto disk = getStoragePolicy()->getDiskByName(name); - if (!disk) - throw Exception("Disk " + name + " does not exists on policy " + getStoragePolicy()->getName(), ErrorCodes::UNKNOWN_DISK); - parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr) { return part_ptr->volume->getDisk()->getName() == disk->getName(); @@ -4112,10 +4110,10 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules( SpacePtr destination_ptr = getDestinationForMoveTTL(*move_ttl_entry, is_insert); if (!destination_ptr) { - if (move_ttl_entry->destination_type == DataDestinationType::VOLUME) + if (move_ttl_entry->destination_type == DataDestinationType::VOLUME && !move_ttl_entry->if_exists) LOG_WARNING(log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but volume was not found or rule is not applicable at the moment", move_ttl_entry->destination_name, log_name); - else if (move_ttl_entry->destination_type == DataDestinationType::DISK) + else if (move_ttl_entry->destination_type == DataDestinationType::DISK && !move_ttl_entry->if_exists) LOG_WARNING(log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but disk was not found or rule is not applicable at the moment", move_ttl_entry->destination_name, log_name); } @@ -4149,7 +4147,7 @@ SpacePtr MergeTreeData::getDestinationForMoveTTL(const TTLDescription & move_ttl auto policy = getStoragePolicy(); if (move_ttl.destination_type == DataDestinationType::VOLUME) { - auto volume = policy->getVolumeByName(move_ttl.destination_name); + auto volume = policy->tryGetVolumeByName(move_ttl.destination_name); if (!volume) return {}; @@ -4161,7 +4159,8 @@ SpacePtr MergeTreeData::getDestinationForMoveTTL(const TTLDescription & move_ttl } else if (move_ttl.destination_type == DataDestinationType::DISK) { - auto disk = policy->getDiskByName(move_ttl.destination_name); + auto disk = policy->tryGetDiskByName(move_ttl.destination_name); + if (!disk) return {}; diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 69303264482..ccf924f2827 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -112,6 +112,7 @@ TTLDescription::TTLDescription(const TTLDescription & other) , aggregate_descriptions(other.aggregate_descriptions) , destination_type(other.destination_type) , destination_name(other.destination_name) + , if_exists(other.if_exists) , recompression_codec(other.recompression_codec) { if (other.expression) @@ -149,6 +150,7 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) aggregate_descriptions = other.aggregate_descriptions; destination_type = other.destination_type; destination_name = other.destination_name; + if_exists = other.if_exists; if (other.recompression_codec) recompression_codec = other.recompression_codec->clone(); @@ -185,9 +187,10 @@ TTLDescription TTLDescription::getTTLFromAST( } else /// rows TTL { + result.mode = ttl_element->mode; result.destination_type = ttl_element->destination_type; result.destination_name = ttl_element->destination_name; - result.mode = ttl_element->mode; + result.if_exists = ttl_element->if_exists; if (ttl_element->mode == TTLMode::DELETE) { diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 17020392013..8f60eb604b5 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -75,6 +75,10 @@ struct TTLDescription /// Name of destination disk or volume String destination_name; + /// If true, do nothing if DISK or VOLUME doesn't exist . + /// Only valid for table MOVE TTLs. + bool if_exists = false; + /// Codec name which will be used to recompress data ASTPtr recompression_codec; diff --git a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml index a76e984e4e6..ae1dc9dd038 100644 --- a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml @@ -76,6 +76,14 @@ + + +
+ jbod1 +
+
+
+
diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 254447478f9..d8373ccb48a 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1174,3 +1174,57 @@ def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) except: pass + + +@pytest.mark.parametrize("name,dest_type", [ + pytest.param("replicated_mt_move_if_exists", "DISK", id="replicated_disk"), + pytest.param("replicated_mt_move_if_exists", "VOLUME", id="replicated_volume"), +]) +def test_ttl_move_if_exists(started_cluster, name, dest_type): + name = unique_table_name(name) + + try: + query_template = """ + CREATE TABLE {name} ( + s1 String, + d1 DateTime + ) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_mt_move_if_exists', '{node_name}') + ORDER BY tuple() + TTL d1 TO {dest_type} {if_exists} 'external' + SETTINGS storage_policy='{policy}' + """ + + with pytest.raises(QueryRuntimeException): + node1.query(query_template.format( \ + name=name, node_name=node1.name, dest_type=dest_type, \ + if_exists='', policy='only_jbod_1')) + + for (node, policy) in zip([node1, node2], ['only_jbod_1', 'small_jbod_with_external']): + node.query(query_template.format( \ + name=name, node_name=node.name, dest_type=dest_type, \ + if_exists='IF EXISTS', policy=policy)) + + data = [] # 10MB in total + for i in range(10): + data.append(("randomPrintableASCII(1024*1024)", "toDateTime({})".format(time.time() - 1))) + + node1.query("INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) + node2.query("SYSTEM SYNC REPLICA {}".format(name)) + + time.sleep(5) + + used_disks1 = get_used_disks_for_table(node1, name) + assert set(used_disks1) == {"jbod1"} + + used_disks2 = get_used_disks_for_table(node2, name) + assert set(used_disks2) == {"external"} + + assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" + assert node2.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" + + finally: + try: + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) + node2.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) + except: + pass diff --git a/tests/queries/0_stateless/02207_ttl_move_if_exists.reference b/tests/queries/0_stateless/02207_ttl_move_if_exists.reference new file mode 100644 index 00000000000..bedef1a5ceb --- /dev/null +++ b/tests/queries/0_stateless/02207_ttl_move_if_exists.reference @@ -0,0 +1 @@ +CREATE TABLE default.t_ttl_move_if_exists\n(\n `d` DateTime,\n `a` UInt32\n)\nENGINE = MergeTree\nORDER BY tuple()\nTTL d TO DISK IF EXISTS \'non_existing_disk\'\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02207_ttl_move_if_exists.sql b/tests/queries/0_stateless/02207_ttl_move_if_exists.sql new file mode 100644 index 00000000000..ab17d343e49 --- /dev/null +++ b/tests/queries/0_stateless/02207_ttl_move_if_exists.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS t_ttl_move_if_exists; + +CREATE TABLE t_ttl_move_if_exists (d DateTime, a UInt32) +ENGINE = MergeTree ORDER BY tuple() +TTL d TO DISK IF EXISTS 'non_existing_disk'; + +SHOW CREATE TABLE t_ttl_move_if_exists; + +DROP TABLE IF EXISTS t_ttl_move_if_exists; From 47412c9619aca0833fadf15a85451a30f05e36b0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Feb 2022 19:31:02 +0000 Subject: [PATCH 081/164] Fixed unit tests --- src/Access/ContextAccess.cpp | 3 ++- src/Access/tests/gtest_access_rights_ops.cpp | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 400ee55a35d..744c3571175 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -425,6 +425,7 @@ bool ContextAccess::checkAccessImplHelper(const AccessFlags & flags, const Args | AccessType::TRUNCATE; const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY; + const AccessFlags function_ddl = AccessType::CREATE_FUNCTION | AccessType::DROP_FUNCTION; const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl; const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE; const AccessFlags write_dcl_access = AccessType::ACCESS_MANAGEMENT - AccessType::SHOW_ACCESS; @@ -432,7 +433,7 @@ bool ContextAccess::checkAccessImplHelper(const AccessFlags & flags, const Args const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; const AccessFlags not_readonly_1_flags = AccessType::CREATE_TEMPORARY_TABLE; - const AccessFlags ddl_flags = table_ddl | dictionary_ddl; + const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl; const AccessFlags introspection_flags = AccessType::INTROSPECTION; }; static const PrecalculatedFlags precalc; diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 2881825dd17..2b4fbf411fe 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -45,7 +45,7 @@ TEST(AccessRights, Union) lhs.grant(AccessType::INSERT); rhs.grant(AccessType::ALL, "db1"); lhs.makeUnion(rhs); - ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, CREATE FUNCTION, DROP, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*"); + ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*"); } From 13cbf79ecb1d75aa80c1938b3ecdec5a8afadf8a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Feb 2022 20:06:23 +0000 Subject: [PATCH 082/164] Improve performance of insert into table functions URL, S3, File, HDFS --- src/Common/ArenaUtils.h | 20 ++++++ src/Coordination/SnapshotableHashTable.h | 16 +---- src/Dictionaries/CacheDictionaryStorage.h | 2 +- src/Dictionaries/DictionaryHelpers.h | 11 ---- src/Dictionaries/FlatDictionary.cpp | 3 +- src/Dictionaries/HashedArrayDictionary.cpp | 1 + src/Dictionaries/HashedDictionary.cpp | 1 + src/Dictionaries/RangeHashedDictionary.cpp | 2 + src/Dictionaries/SSDCacheDictionaryStorage.h | 1 + src/Storages/PartitionedSink.cpp | 65 ++++++++++++-------- src/Storages/PartitionedSink.h | 11 +++- 11 files changed, 80 insertions(+), 53 deletions(-) create mode 100644 src/Common/ArenaUtils.h diff --git a/src/Common/ArenaUtils.h b/src/Common/ArenaUtils.h new file mode 100644 index 00000000000..0a588692367 --- /dev/null +++ b/src/Common/ArenaUtils.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include +#include + +/** Copy string value into Arena. + * Arena should support method: + * char * alloc(size_t size). + */ +template +inline StringRef copyStringInArena(Arena & arena, StringRef value) +{ + size_t key_size = value.size; + char * place_for_key = arena.alloc(key_size); + memcpy(reinterpret_cast(place_for_key), reinterpret_cast(value.data), key_size); + StringRef result{place_for_key, key_size}; + + return result; +} diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index b1d72578530..7112e05f07e 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -113,17 +114,6 @@ private: } } - StringRef copyStringInArena(const std::string & value_to_copy) - { - size_t value_to_copy_size = value_to_copy.size(); - char * place_for_key = arena.alloc(value_to_copy_size); - memcpy(reinterpret_cast(place_for_key), reinterpret_cast(value_to_copy.data()), value_to_copy_size); - StringRef updated_value{place_for_key, value_to_copy_size}; - - return updated_value; - } - - public: using iterator = typename List::iterator; @@ -137,7 +127,7 @@ public: if (!it) { - ListElem elem{copyStringInArena(key), value, true}; + ListElem elem{copyStringInArena(arena, key), value, true}; auto itr = list.insert(list.end(), elem); bool inserted; map.emplace(itr->key, it, inserted, hash_value); @@ -159,7 +149,7 @@ public: if (it == map.end()) { - ListElem elem{copyStringInArena(key), value, true}; + ListElem elem{copyStringInArena(arena, key), value, true}; auto itr = list.insert(list.end(), elem); bool inserted; map.emplace(itr->key, it, inserted, hash_value); diff --git a/src/Dictionaries/CacheDictionaryStorage.h b/src/Dictionaries/CacheDictionaryStorage.h index d6d04075a3d..566515c7cc8 100644 --- a/src/Dictionaries/CacheDictionaryStorage.h +++ b/src/Dictionaries/CacheDictionaryStorage.h @@ -8,10 +8,10 @@ #include #include #include +#include #include #include #include -#include namespace DB diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 5c2b6b27afd..f2d7febfa8e 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -623,17 +623,6 @@ void mergeBlockWithPipe( } } -template -static StringRef copyStringInArena(Arena & arena, StringRef value) -{ - size_t key_size = value.size; - char * place_for_key = arena.alloc(key_size); - memcpy(reinterpret_cast(place_for_key), reinterpret_cast(value.data), key_size); - StringRef result{place_for_key, key_size}; - - return result; -} - /** * Returns ColumnVector data as PaddedPodArray. diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 40cc735557c..0c82da7b73b 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -13,7 +14,7 @@ #include #include -#include +#include #include #include diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index e35340c7618..ea041c63d73 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -1,5 +1,6 @@ #include "HashedArrayDictionary.h" +#include #include #include #include diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index c83735a6330..b70f018df6b 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -1,5 +1,6 @@ #include "HashedDictionary.h" +#include #include #include #include diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 14c8fc7c749..5330bc684c3 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -1,5 +1,7 @@ #include +#include + #include #include #include diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index 292e60f17f9..adbe4084d81 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 5e8f2a9e132..a42ea4ceff6 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -1,5 +1,7 @@ #include "PartitionedSink.h" +#include + #include #include @@ -40,19 +42,18 @@ PartitionedSink::PartitionedSink( } -SinkPtr PartitionedSink::getSinkForPartition(const String & partition_id) +SinkPtr PartitionedSink::getSinkForPartitionKey(StringRef partition_key) { - auto it = sinks.find(partition_id); - if (it == sinks.end()) + auto it = partition_id_to_sink.find(partition_key); + if (it == partition_id_to_sink.end()) { - auto sink = createSinkForPartition(partition_id); - std::tie(it, std::ignore) = sinks.emplace(partition_id, sink); + auto sink = createSinkForPartition(partition_key.toString()); + std::tie(it, std::ignore) = partition_id_to_sink.emplace(partition_key, sink); } return it->second; } - void PartitionedSink::consume(Chunk chunk) { const auto & columns = chunk.getColumns(); @@ -61,45 +62,59 @@ void PartitionedSink::consume(Chunk chunk) block_with_partition_by_expr.setColumns(columns); partition_by_expr->execute(block_with_partition_by_expr); - const auto * column = block_with_partition_by_expr.getByName(partition_by_column_name).column.get(); + const auto * partition_by_result_column = block_with_partition_by_expr.getByName(partition_by_column_name).column.get(); - std::unordered_map sub_chunks_indices; - IColumn::Selector selector; - for (size_t row = 0; row < chunk.getNumRows(); ++row) + size_t chunk_rows = chunk.getNumRows(); + chunk_row_index_to_partition_index.resize(chunk_rows); + + partition_id_to_chunk_index.clear(); + + for (size_t row = 0; row < chunk_rows; ++row) { - auto value = column->getDataAt(row); - auto [it, inserted] = sub_chunks_indices.emplace(value, sub_chunks_indices.size()); - selector.push_back(it->second); + auto partition_key = partition_by_result_column->getDataAt(row); + auto [it, inserted] = partition_id_to_chunk_index.insert(makePairNoInit(partition_key, partition_id_to_chunk_index.size())); + if (inserted) + it->value.first = copyStringInArena(partition_keys_arena, partition_key); + + chunk_row_index_to_partition_index[row] = it->getMapped(); } - Chunks sub_chunks; - sub_chunks.reserve(sub_chunks_indices.size()); - for (size_t column_index = 0; column_index < columns.size(); ++column_index) + size_t columns_size = columns.size(); + size_t partitions_size = partition_id_to_chunk_index.size(); + + Chunks partition_index_to_chunk; + partition_index_to_chunk.reserve(partitions_size); + + for (size_t column_index = 0; column_index < columns_size; ++column_index) { - MutableColumns column_sub_chunks = columns[column_index]->scatter(sub_chunks_indices.size(), selector); - if (column_index == 0) /// Set sizes for sub-chunks. + MutableColumns partition_index_to_column_split = columns[column_index]->scatter(partitions_size, chunk_row_index_to_partition_index); + + /// Add chunks into partition_index_to_chunk with sizes of result columns + if (column_index == 0) { - for (const auto & column_sub_chunk : column_sub_chunks) + for (const auto & partition_column : partition_index_to_column_split) { - sub_chunks.emplace_back(Columns(), column_sub_chunk->size()); + partition_index_to_chunk.emplace_back(Columns(), partition_column->size()); } } - for (size_t sub_chunk_index = 0; sub_chunk_index < column_sub_chunks.size(); ++sub_chunk_index) + + for (size_t partition_index = 0; partition_index < partitions_size; ++partition_index) { - sub_chunks[sub_chunk_index].addColumn(std::move(column_sub_chunks[sub_chunk_index])); + partition_index_to_chunk[partition_index].addColumn(std::move(partition_index_to_column_split[partition_index])); } } - for (const auto & [partition_id, sub_chunk_index] : sub_chunks_indices) + for (const auto & [partition_key, partition_index] : partition_id_to_chunk_index) { - getSinkForPartition(partition_id)->consume(std::move(sub_chunks[sub_chunk_index])); + auto sink = getSinkForPartitionKey(partition_key); + sink->consume(std::move(partition_index_to_chunk[partition_index])); } } void PartitionedSink::onFinish() { - for (auto & [partition_id, sink] : sinks) + for (auto & [_, sink] : partition_id_to_sink) { sink->onFinish(); } diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index bc59a603fac..7ed29f1b197 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -1,5 +1,8 @@ #pragma once +#include +#include +#include #include #include #include @@ -34,9 +37,13 @@ private: ExpressionActionsPtr partition_by_expr; String partition_by_column_name; - std::unordered_map sinks; + absl::flat_hash_map partition_id_to_sink; + HashMapWithSavedHash partition_id_to_chunk_index; + IColumn::Selector chunk_row_index_to_partition_index; + Arena partition_keys_arena; + + SinkPtr getSinkForPartitionKey(StringRef partition_key); - SinkPtr getSinkForPartition(const String & partition_id); }; } From 9e893dfda7ec988e110df621733535f2bb9e58c3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Feb 2022 23:35:03 +0300 Subject: [PATCH 083/164] fixes --- src/Databases/DatabaseOnDisk.cpp | 4 + src/Interpreters/InterpreterCreateQuery.cpp | 109 ++++++++++-------- src/Interpreters/InterpreterCreateQuery.h | 2 + src/Parsers/ParserCreateQuery.cpp | 24 ++-- .../02184_default_table_engine.reference | 14 +++ .../02184_default_table_engine.sql | 77 ++++++++++++- .../02184_table_engine_access.reference | 2 + .../0_stateless/02184_table_engine_access.sh | 23 ++++ 8 files changed, 190 insertions(+), 65 deletions(-) create mode 100644 tests/queries/0_stateless/02184_table_engine_access.reference create mode 100755 tests/queries/0_stateless/02184_table_engine_access.sh diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 29591a5f88f..9f56b5f7676 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -77,6 +77,10 @@ std::pair createTableFromAST( /// - the code is simpler, since the query is already brought to a suitable form. if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns) { + if (!ast_create_query.storage || !ast_create_query.storage->engine) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid storage definition in metadata file: " + "it's a bug or result of manual intervention in metadata files"); + if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(ast_create_query.storage->engine->name)) throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); /// Leave columns empty. diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 96a494068e5..ac415214892 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -93,6 +93,7 @@ namespace ErrorCodes extern const int UNKNOWN_DATABASE; extern const int PATH_ACCESS_DENIED; extern const int NOT_IMPLEMENTED; + extern const int ENGINE_REQUIRED; } namespace fs = std::filesystem; @@ -159,6 +160,9 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage)); } + if (create.storage && !create.storage->engine) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Database engine must be specified"); + if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated" || create.storage->engine->name == "MaterializedPostgreSQL") @@ -583,6 +587,17 @@ ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const A InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTablePropertiesAndNormalizeCreateQuery(ASTCreateQuery & create) const { + /// Set the table engine if it was not specified explicitly. + setEngine(create); + + /// We have to check access rights again (in case engine was changed). + if (create.storage) + { + auto source_access_type = StorageFactory::instance().getSourceAccessType(create.storage->engine->name); + if (source_access_type != AccessType::NONE) + getContext()->checkAccess(source_access_type); + } + TableProperties properties; TableLockHolder as_storage_lock; @@ -665,8 +680,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti create.columns_list->setOrReplace(create.columns_list->projections, new_projections); validateTableStructure(create, properties); - /// Set the table engine if it was not specified explicitly. - setEngine(create); assert(as_database_saved.empty() && as_table_saved.empty()); std::swap(create.as_database, as_database_saved); @@ -750,25 +763,39 @@ String InterpreterCreateQuery::getTableEngineName(DefaultTableEngine default_tab } } +void InterpreterCreateQuery::setDefaultTableEngine(ASTStorage & storage, ContextPtr local_context) +{ + if (local_context->getSettingsRef().default_table_engine.value == DefaultTableEngine::None) + throw Exception(ErrorCodes::ENGINE_REQUIRED, "Table engine is not specified in CREATE query"); + + auto engine_ast = std::make_shared(); + auto default_table_engine = local_context->getSettingsRef().default_table_engine.value; + engine_ast->name = getTableEngineName(default_table_engine); + engine_ast->no_empty_args = true; + storage.set(storage.engine, engine_ast); +} + void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { if (create.as_table_function) return; - if (create.temporary && create.storage && !create.storage->engine) - throw Exception("Storage is defined without ENGINE for temprorary table.", ErrorCodes::LOGICAL_ERROR); + if (create.is_dictionary || create.is_ordinary_view || create.is_live_view) + return; - if (create.temporary && create.storage && create.storage->engine) - { - if (create.storage->engine->name == "Memory") - return; - - throw Exception(ErrorCodes::INCORRECT_QUERY, - "Temporary tables can only be created with ENGINE = Memory, not {}", create.storage->engine->name); - } - if (create.temporary) { + if (create.storage && create.storage->engine && create.storage->engine->name != "Memory") + throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables can only be created with ENGINE = Memory, not {}", + create.storage->engine->name); + + /// It's possible if some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not. + /// It makes sense when default_table_engine setting is used, but not for temporary tables. + /// For temporary tables we ignore this setting to allow CREATE TEMPORARY TABLE query without specifying ENGINE + /// even if setting is set to MergeTree or something like that (otherwise MergeTree will be substituted and query will fail). + if (create.storage && !create.storage->engine) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Invalid storage definition for temporary table: must be either ENGINE = Memory or empty"); + auto engine_ast = std::make_shared(); engine_ast->name = "Memory"; engine_ast->no_empty_args = true; @@ -777,7 +804,15 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const create.set(create.storage, storage_ast); return; } - + + if (create.storage) + { + /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. + if (!create.storage->engine) + setDefaultTableEngine(*create.storage, getContext()); + return; + } + if (!create.as_table.empty()) { /// NOTE Getting the structure from the table specified in the AS is done not atomically with the creation of the table. @@ -791,49 +826,29 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const const String qualified_name = backQuoteIfNeed(as_database_name) + "." + backQuoteIfNeed(as_table_name); if (as_create.is_ordinary_view) - throw Exception( - "Cannot CREATE a table AS " + qualified_name + ", it is a View", - ErrorCodes::INCORRECT_QUERY); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot CREATE a table AS {}, it is a View", qualified_name); if (as_create.is_live_view) - throw Exception( - "Cannot CREATE a table AS " + qualified_name + ", it is a Live View", - ErrorCodes::INCORRECT_QUERY); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot CREATE a table AS {}, it is a Live View", qualified_name); if (as_create.is_window_view) - throw Exception( - "Cannot CREATE a table AS " + qualified_name + ", it is a Window View", - ErrorCodes::INCORRECT_QUERY); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot CREATE a table AS {}, it is a Window View", qualified_name); if (as_create.is_dictionary) - throw Exception( - "Cannot CREATE a table AS " + qualified_name + ", it is a Dictionary", - ErrorCodes::INCORRECT_QUERY); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot CREATE a table AS {}, it is a Dictionary", qualified_name); + + if (as_create.storage) + create.set(create.storage, as_create.storage->ptr()); + else if (as_create.as_table_function) + create.as_table_function = as_create.as_table_function->clone(); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot set engine, it's a bug."); - if (!create.storage) - { - if (as_create.storage) - create.set(create.storage, as_create.storage->ptr()); - else if (as_create.as_table_function) - create.as_table_function = as_create.as_table_function->clone(); - } return; } - if (create.is_ordinary_view || create.is_live_view || create.is_window_view || create.is_dictionary) - return; - - if (!create.storage) - create.set(create.storage, std::make_shared()); - - if (!create.storage->engine && getContext()->getSettingsRef().default_table_engine.value != DefaultTableEngine::None) - { - auto engine_ast = std::make_shared(); - auto default_table_engine = getContext()->getSettingsRef().default_table_engine.value; - engine_ast->name = getTableEngineName(default_table_engine); - engine_ast->no_empty_args = true; - create.storage->set(create.storage->engine, engine_ast); - } + create.set(create.storage, std::make_shared()); + setDefaultTableEngine(*create.storage, getContext()); } static void generateUUIDForTable(ASTCreateQuery & create) diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index ec4c41825da..5804d817fe2 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -16,6 +16,7 @@ namespace DB class ASTCreateQuery; class ASTExpressionList; class ASTConstraintDeclaration; +class ASTStorage; class IDatabase; using DatabasePtr = std::shared_ptr; @@ -83,6 +84,7 @@ private: TableProperties getTablePropertiesAndNormalizeCreateQuery(ASTCreateQuery & create) const; void validateTableStructure(const ASTCreateQuery & create, const TableProperties & properties) const; static String getTableEngineName(DefaultTableEngine default_table_engine); + static void setDefaultTableEngine(ASTStorage & storage, ContextPtr local_context); void setEngine(ASTCreateQuery & create) const; AccessRightsElements getRequiredAccess() const; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 8c34bb4cbc3..9c9989dc39f 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -573,20 +573,10 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (!storage_parse_result && !is_temporary) { if (s_as.ignore(pos, expected) && !table_function_p.parse(pos, as_table_function, expected)) - { return false; - } - else - { - // ENGINE can be omitted if default_table_engine is set. - // Need to check in Interpreter - if (!storage) - { - auto storage_ast = std::make_shared(); - storage = storage_ast; - } - } } + + /// Will set default table engine if Storage clause was not parsed } /** Create queries without list of columns: * - CREATE|ATTACH TABLE ... AS ... @@ -652,12 +642,14 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (comment) query->set(query->comment, comment); - if (query->storage && query->columns_list && query->columns_list->primary_key) + if (query->columns_list && query->columns_list->primary_key) { - if (query->storage->primary_key) - { + /// If engine is not set will use default one + if (!query->storage) + query->set(query->storage, std::make_shared()); + else if (query->storage->primary_key) throw Exception("Multiple primary keys are not allowed.", ErrorCodes::BAD_ARGUMENTS); - } + query->storage->primary_key = query->columns_list->primary_key; } diff --git a/tests/queries/0_stateless/02184_default_table_engine.reference b/tests/queries/0_stateless/02184_default_table_engine.reference index daa97739757..200578f3da9 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.reference +++ b/tests/queries/0_stateless/02184_default_table_engine.reference @@ -9,5 +9,19 @@ CREATE TABLE default.numbers1\n(\n `number` UInt64\n)\nENGINE = Memory CREATE TABLE default.numbers2\n(\n `number` UInt64\n)\nENGINE = MergeTree\nORDER BY intHash32(number)\nSAMPLE BY intHash32(number)\nSETTINGS index_granularity = 8192 45 CREATE TABLE default.numbers3\n(\n `number` UInt64\n)\nENGINE = Log +CREATE MATERIALIZED VIEW default.test_view_filtered\n(\n `EventDate` Date,\n `CounterID` UInt32\n)\nENGINE = Memory AS\nSELECT\n CounterID,\n EventDate\nFROM default.test_table\nWHERE EventDate < \'2013-01-01\' 2014-01-02 0 0 1970-01-01 03:00:00 2014-01-02 03:04:06 1 2014-01-02 07:04:06 +CREATE TABLE default.t1\n(\n `Rows` UInt64,\n `MaxHitTime` DateTime(\'UTC\')\n)\nENGINE = MergeTree\nORDER BY Rows\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t2\n(\n `Rows` UInt64,\n `MaxHitTime` DateTime(\'UTC\')\n)\nENGINE = Memory +CREATE TABLE default.mt\n(\n `a` UInt64,\n `b` Nullable(String),\n INDEX b_index b TYPE set(123) GRANULARITY 1\n)\nENGINE = MergeTree\nPRIMARY KEY (a, coalesce(b, \'test\'))\nORDER BY (a, coalesce(b, \'test\'))\nSETTINGS index_granularity = 8192 +CREATE TABLE default.mt2\n(\n `a` UInt64,\n `b` Nullable(String),\n INDEX b_index b TYPE set(123) GRANULARITY 1\n)\nENGINE = MergeTree\nPRIMARY KEY (a, coalesce(b, \'test\'))\nORDER BY (a, coalesce(b, \'test\'))\nSETTINGS index_granularity = 8192 +CREATE TEMPORARY TABLE tmp\n(\n `n` Int32\n)\nENGINE = Memory +CREATE TABLE default.log\n(\n `n` Int32\n)\nENGINE = Log +CREATE TABLE default.log1\n(\n `n` Int32\n)\nENGINE = Log +CREATE TABLE default.mem\n(\n `n` Int32\n)\nENGINE = Memory +CREATE TABLE default.mt\n(\n `n` UInt8\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.mem\n(\n `n` UInt8\n)\nENGINE = Memory +CREATE TABLE default.val\n(\n `n` Int32\n) AS values(\'n int\', 1, 2) +CREATE TABLE default.val2\n(\n `n` Int32\n) AS values(\'n int\', 1, 2) +CREATE TABLE default.log\n(\n `n` Int32\n)\nENGINE = Log diff --git a/tests/queries/0_stateless/02184_default_table_engine.sql b/tests/queries/0_stateless/02184_default_table_engine.sql index 496e79de23a..d129ccc801e 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.sql +++ b/tests/queries/0_stateless/02184_default_table_engine.sql @@ -3,11 +3,13 @@ SET default_table_engine = 'Log'; CREATE TABLE table_02184 (x UInt8); SHOW CREATE TABLE table_02184; DROP TABLE table_02184; + SET default_table_engine = 'MergeTree'; CREATE TABLE table_02184 (x UInt8); --{serverError 42} CREATE TABLE table_02184 (x UInt8, PRIMARY KEY (x)); SHOW CREATE TABLE table_02184; DROP TABLE table_02184; + CREATE TABLE test_optimize_exception (date Date) PARTITION BY toYYYYMM(date) ORDER BY date; SHOW CREATE TABLE test_optimize_exception; DROP TABLE test_optimize_exception; @@ -15,6 +17,7 @@ CREATE TABLE table_02184 (x UInt8) PARTITION BY x; --{serverError 36} CREATE TABLE table_02184 (x UInt8) ORDER BY x; SHOW CREATE TABLE table_02184; DROP TABLE table_02184; + CREATE TABLE table_02184 (x UInt8) PRIMARY KEY x; SHOW CREATE TABLE table_02184; DROP TABLE table_02184; @@ -23,25 +26,95 @@ CREATE TABLE numbers1 AS SELECT number FROM numbers(10); SHOW CREATE TABLE numbers1; SELECT avg(number) FROM numbers1; DROP TABLE numbers1; + SET default_table_engine = 'MergeTree'; CREATE TABLE numbers2 ORDER BY intHash32(number) SAMPLE BY intHash32(number) AS SELECT number FROM numbers(10); SELECT sum(number) FROM numbers2; SHOW CREATE TABLE numbers2; DROP TABLE numbers2; + CREATE TABLE numbers3 ENGINE = Log AS SELECT number FROM numbers(10); SELECT sum(number) FROM numbers3; SHOW CREATE TABLE numbers3; DROP TABLE numbers3; + CREATE TABLE test_table (EventDate Date, CounterID UInt32, UserID UInt64, EventTime DateTime('Europe/Moscow'), UTCEventTime DateTime('UTC')) PARTITION BY EventDate PRIMARY KEY CounterID; SET default_table_engine = 'Memory'; CREATE MATERIALIZED VIEW test_view (Rows UInt64, MaxHitTime DateTime('Europe/Moscow')) AS SELECT count() AS Rows, max(UTCEventTime) AS MaxHitTime FROM test_table; CREATE MATERIALIZED VIEW test_view_filtered (EventDate Date, CounterID UInt32) POPULATE AS SELECT CounterID, EventDate FROM test_table WHERE EventDate < '2013-01-01'; +SHOW CREATE TABLE test_view_filtered; INSERT INTO test_table (EventDate, UTCEventTime) VALUES ('2014-01-02', '2014-01-02 03:04:06'); SELECT * FROM test_table; SELECT * FROM test_view; SELECT * FROM test_view_filtered; -DROP TABLE test_table; DROP TABLE test_view; -DROP TABLE test_view_filtered; \ No newline at end of file +DROP TABLE test_view_filtered; + +SET default_table_engine = 'MergeTree'; +CREATE MATERIALIZED VIEW test_view ORDER BY Rows AS SELECT count() AS Rows, max(UTCEventTime) AS MaxHitTime FROM test_table; +SET default_table_engine = 'Memory'; +CREATE TABLE t1 AS test_view; +CREATE TABLE t2 ENGINE=Memory AS test_view; +SHOW CREATE TABLE t1; +SHOW CREATE TABLE t2; +DROP TABLE test_view; +DROP TABLE test_table; +DROP TABLE t1; +DROP TABLE t2; + + +CREATE DATABASE test_02184 ORDER BY kek; -- {serverError 80} +CREATE DATABASE test_02184 SETTINGS x=1; -- {serverError 80} +CREATE TABLE table_02184 (x UInt8, y int, PRIMARY KEY (x)) ENGINE=MergeTree PRIMARY KEY y; -- {clientError 36} +SET default_table_engine = 'MergeTree'; +CREATE TABLE table_02184 (x UInt8, y int, PRIMARY KEY (x)) PRIMARY KEY y; -- {clientError 36} + +CREATE TABLE mt (a UInt64, b Nullable(String), PRIMARY KEY (a, coalesce(b, 'test')), INDEX b_index b TYPE set(123) GRANULARITY 1); +SHOW CREATE TABLE mt; +SET default_table_engine = 'Log'; +CREATE TABLE mt2 AS mt; +SHOW CREATE TABLE mt2; +DROP TABLE mt; + +SET default_table_engine = 'Log'; +CREATE TEMPORARY TABLE tmp (n int); +SHOW CREATE TEMPORARY TABLE tmp; +CREATE TEMPORARY TABLE tmp1 (n int) ENGINE=Memory; +CREATE TEMPORARY TABLE tmp2 (n int) ENGINE=Log; -- {serverError 80} +CREATE TEMPORARY TABLE tmp2 (n int) ORDER BY n; -- {serverError 80} +CREATE TEMPORARY TABLE tmp2 (n int, PRIMARY KEY (n)); -- {serverError 80} + +CREATE TABLE log (n int); +SHOW CREATE log; +SET default_table_engine = 'MergeTree'; +CREATE TABLE log1 AS log; +SHOW CREATE log1; +CREATE TABLE mem AS log1 ENGINE=Memory; +SHOW CREATE mem; +DROP TABLE log; +DROP TABLE log1; +DROP TABLE mem; + +SET default_table_engine = 'None'; +CREATE TABLE mem AS SELECT 1 as n; --{serverError 119} +SET default_table_engine = 'Memory'; +CREATE TABLE mem ORDER BY n AS SELECT 1 as n; -- {serverError 36} +SET default_table_engine = 'MergeTree'; +CREATE TABLE mt ORDER BY n AS SELECT 1 as n; +CREATE TABLE mem ENGINE=Memory AS SELECT 1 as n; +SHOW CREATE TABLE mt; +SHOW CREATE TABLE mem; +DROP TABLE mt; +DROP TABLE mem; + +CREATE TABLE val AS values('n int', 1, 2); +CREATE TABLE val2 AS val; +CREATE TABLE log ENGINE=Log AS val; +SHOW CREATE TABLE val; +SHOW CREATE TABLE val2; +SHOW CREATE TABLE log; +DROP TABLE val; +DROP TABLE val2; +DROP TABLE log; diff --git a/tests/queries/0_stateless/02184_table_engine_access.reference b/tests/queries/0_stateless/02184_table_engine_access.reference new file mode 100644 index 00000000000..99a67d4daa4 --- /dev/null +++ b/tests/queries/0_stateless/02184_table_engine_access.reference @@ -0,0 +1,2 @@ +ACCESS_DENIED +CREATE TABLE default.t\n(\n `line` String\n)\nENGINE = URL(\'https://clickhouse.com\', \'LineAsString\') diff --git a/tests/queries/0_stateless/02184_table_engine_access.sh b/tests/queries/0_stateless/02184_table_engine_access.sh new file mode 100755 index 00000000000..dbbf28e46d4 --- /dev/null +++ b/tests/queries/0_stateless/02184_table_engine_access.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +# Tags: no-parallel +# Tag no-parallel: create user + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS user_test_02184;" +$CLICKHOUSE_CLIENT --query "CREATE USER user_test_02184 IDENTIFIED WITH plaintext_password BY 'user_test_02184';" +${CLICKHOUSE_CLIENT} -q "REVOKE ALL ON *.* FROM user_test_02184" + +$CLICKHOUSE_CLIENT --query "GRANT CREATE ON *.* TO user_test_02184;" + +$CLICKHOUSE_CLIENT --query "CREATE TABLE url ENGINE=URL('https://clickhouse.com', LineAsString)" + +$CLICKHOUSE_CLIENT --user=user_test_02184 --password=user_test_02184 --query "CREATE TABLE t AS url" 2>&1| grep -Fo "ACCESS_DENIED" | uniq + +$CLICKHOUSE_CLIENT --query "GRANT URL ON *.* TO user_test_02184;" +$CLICKHOUSE_CLIENT --user=user_test_02184 --password=user_test_02184 --query "CREATE TABLE t AS url" +$CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE t" +$CLICKHOUSE_CLIENT --query "DROP TABLE t" +$CLICKHOUSE_CLIENT --query "DROP TABLE url" From 691cb3352b607b5c028be2481d7f730d8d85fb2f Mon Sep 17 00:00:00 2001 From: Tom Risse Date: Thu, 10 Feb 2022 14:17:31 -0800 Subject: [PATCH 084/164] Add support agreement page and snippets. (#34512) --- website/support/agreement/index.html | 27 ++++ .../templates/support/agreement-content.html | 120 ++++++++++++++++++ website/templates/support/agreement-hero.html | 10 ++ 3 files changed, 157 insertions(+) create mode 100644 website/support/agreement/index.html create mode 100644 website/templates/support/agreement-content.html create mode 100644 website/templates/support/agreement-hero.html diff --git a/website/support/agreement/index.html b/website/support/agreement/index.html new file mode 100644 index 00000000000..59e5ca9ab75 --- /dev/null +++ b/website/support/agreement/index.html @@ -0,0 +1,27 @@ +{% set prefetch_items = [ + ('/docs/en/', 'document') +] %} + +{% extends "templates/base.html" %} + +{% block extra_meta %} + +{% include "templates/common_fonts.html" %} +{% endblock %} + +{% block nav %} + +{% include "templates/global/nav.html" %} + +{% endblock %} + +{% block content %} + +{% include "templates/support/agreement-hero.html" %} + +{% include "templates/support/agreement-content.html" %} + +{% include "templates/global/newsletter.html" %} +{% include "templates/global/github_stars.html" %} + +{% endblock %} diff --git a/website/templates/support/agreement-content.html b/website/templates/support/agreement-content.html new file mode 100644 index 00000000000..4ca64e69599 --- /dev/null +++ b/website/templates/support/agreement-content.html @@ -0,0 +1,120 @@ +
+
+ +

This ClickHouse Subscription Agreement, including all referenced URLs, which are incorporated herein by reference (collectively, this “Agreement”), is entered into as of the date on which an applicable Order Form is fully executed (“Effective Date”), by and between the ClickHouse entity ("ClickHouse") set forth on such Order Form, and the entity identified thereon as the “Customer” (“Customer”).

+ +

1. DEFINITIONS
Capitalized terms used herein have the meaning ascribed below, or where such terms are first used, as applicable.

+ +

1.1 "Affiliate" means, with respect to a party, any entity that controls, is controlled by, or which is under common control with, such party, where "control" means ownership of at least fifty percent (50%) of the outstanding voting shares of the entity, or the contractual right to establish policy for, and manage the operations of, the entity.

+ +

1.2 "Order Form" means an ordering document provided by ClickHouse pursuant to which Customer purchases Subscriptions under this Agreement.

+ +

1.3 "Qualifying PO" means a purchase order issued by customer for the purpose of purchasing a Subscription, which (i) references the number of an applicable Order Form provided to Customer by ClickHouse and (ii) clearly states the purchase order is subject to the terms and conditions of this Agreement.

+ +

1.4 "Software" means the ClickHouse software of the same name that is licensed for use under the Apache 2.0 license.

+ +

1.5 "Subscription" means Customer's right, for a fixed period of time, to receive Support Services, as set forth in the applicable Order Form.

+ +

1.6 "Subscription Term" means the period of time for which a Subscription is valid, as further described in Section 7.1 of this Agreement.

+ +

1.7 "Support Services" means maintenance and support services for the Software, as more fully described in the Support Services Policy.

+ +

1.8 "Support Services Policy" means ClickHouse's support services policy as further described at https://clickhouse.com/support/policy/. ClickHouse reserves the right to reasonably modify the Support Services Policy during a Subscription Term, provided however, ClickHouse shall not materially diminish the level of Support Services during a Subscription Term. The effective date of each version of the Support Services Policy will be stated thereon, and ClickHouse agrees to archive copies of each version, and make the same available to Customer upon written request (e-mail sufficient). The parties agree that the Support Services Policy is hereby incorporated into these terms and conditions by this reference.

+ +

2. AGREEMENT SCOPE AND PERFORMANCE OF SUPPORT SERVICES

+ +

2.1 Agreement Scope. This Agreement includes terms and conditions applicable to Subscriptions for Support Services purchased under each Order Form entered into by the parties under Section 2.2 below, which Support Services may be used by Customer solely for Internal use and in connection with the use case(s) set forth on the applicable Order Form.

+ +

2.2 Order for Support Services Subscriptions. Orders for Subscriptions may be placed by Customer through (1) the execution of Order Forms with ClickHouse or (2) issuance by Customer of a Qualifying PO, which will be deemed to constitute, for the purposes of this Agreement, the execution by Customer of the referenced Order Form.

+ +

2.3 Affiliates. The parties agree that their respective Affiliates may also conduct business under this Agreement by entering into Order Forms, which in some cases may be subject to such additional and/or alternative terms and conditions to those contained in this Agreement as may be mutually agreed in the Order Form or an attachment thereto, as applicable. Accordingly, where Affiliates of the parties conduct business hereunder, references to Customer herein shall include any applicable Customer Affiliate, and references to ClickHouse herein shall include any applicable ClickHouse Affiliate. The parties agree that where either of them or one of their Affiliates enters into an Order Form with an Affiliate of the other party, that such Affiliate shall be solely responsible for performing all of its obligations under this Agreement in connection with such Order Form.

+ +

2.4 Performance of Support Services. Subject to Customer’s payment of all fees (as set forth in an applicable Order Form), ClickHouse will provide Customer with Support Services for the Software during an applicable Subscription Term in accordance with this Agreement and the Support Services Policy. Customer will reasonably cooperate with ClickHouse in connection with the Support Services, including, without limitation, by providing ClickHouse reasonable remote access to its installations, server cloud (or hosting provider), Software and equipment in connection therewith. Further, Customer will designate appropriately skilled personnel to serve as ClickHouse’s central contacts in connection with the use, operation and support of the Software. Customer understands that ClickHouse’s performance of Support Services is dependent in part on Customer’s cooperation, actions, and performance. ClickHouse shall not be responsible for any delays or interruptions in its performance of Support Services, or any claims arising therefrom, due to Customer’s lack of cooperation or acts or omissions. ClickHouse may use its Affiliates or subcontractors to provide Support Services to Customer, provided that ClickHouse remains responsible to Customer for performance.

+ +

3. PAYMENT AND TAXES

+ +

3.1 Payment. ClickHouse will invoice Customer for the fees due under each Order Form or otherwise under this Agreement, and Customer will pay such fees within thirty (30) days after receipt of an applicable invoice. All invoices will be paid in the currency set forth on the applicable Order Form. Payments will be made without right of set-off or chargeback. Except as otherwise expressly provided in this Agreement, any and all payments made by Customer pursuant to this Agreement or any Order Form are non-refundable, and all commitments to make any payments hereunder or under any Order Form are non-cancellable.

+ +

3.2 Taxes. All fees stated on an Order Form are exclusive of any applicable sales, use, value added and excise taxes levied upon the delivery or use of the taxable components, if any, of any Subscription purchased by Customer under this Agreement (collectively, “Taxes”). Taxes do not include any taxes on the net income of ClickHouse or any of its Affiliates. Unless Customer provides ClickHouse a valid state sales/use/excise tax exemption certificate or Direct Pay Permit, and provided that ClickHouse separately states any such taxes in the applicable invoice, Customer will pay and be solely responsible for all Taxes. If Customer is required by any foreign governmental authority to deduct or withhold any portion of the amount invoiced for the delivery or use of Support Services under this Agreement, Customer shall increase the sum paid to ClickHouse by an amount necessary for the total payment to ClickHouse equal to the amount originally invoiced.

+ +

4. CONFIDENTIAL INFORMATION

+ +

4.1 Confidential Information. Both parties acknowledge that, in the course of performing this Agreement, they may obtain information relating to products (such as goods, services, and software) of the other party, or relating to the parties themselves, which is of a confidential and proprietary nature ("Confidential Information"). Confidential Information includes materials and all communications concerning ClickHouse's or Customer's business and marketing strategies, including but not limited to employee and customer lists, customer profiles, project plans, design documents, product strategies and pricing data, research, advertising plans, leads and sources of supply, development activities, design and coding, interfaces with the Products, anything provided by either party to the other in connection with the Products and/or Support Services provided under this Agreement, including, without limitation, computer programs, technical drawings, algorithms, know-how, formulas, processes, ideas, inventions (whether patentable or not), schematics and other technical plans and other information of the parties which by its nature can be reasonably expected to be proprietary and confidential, whether it is presented in oral, printed, written, graphic or photographic or other tangible form (including information received, stored or transmitted electronically) even though specific designation as Confidential Information has not been made. Confidential Information also includes any notes, summaries, analyses of the foregoing that are prepared by the receiving party.

+ +

4.2 Non-use and Non-disclosure. The parties shall at all times, both during the Term and thereafter keep in trust and confidence all Confidential Information of the other party using commercially reasonable care (but in no event less than the same degree of care that the receiving party uses to protect its own Confidential Information) and shall not use such Confidential Information other than as necessary to carry out its duties under this Agreement, nor shall either party disclose any such Confidential Information to third parties other than to Affiliates or as necessary to carry out its duties under this Agreement without the other party's prior written consent, provided that each party shall be allowed to disclose Confidential Information of the other party to the extent that such disclosure is approved in writing by such other party, or necessary to enforce its rights under this Agreement.

+ +

4.3 Non-Applicability. The obligations of confidentiality shall not apply to information which (i) has entered the public domain or is otherwise publicly available, except where such entry or availability is the result of a party's breach of this Agreement; (ii) prior to disclosure hereunder was already in the receiving party's possession without restriction as evidenced by appropriate documentation; (iii) subsequent to disclosure hereunder is obtained by the receiving party on a non-confidential basis from a third party who has the right to disclose such information; or (iv) was developed by the receiving party without any use of any of the Confidential Information as evidenced by appropriate documentation.

+ +

4.4 Terms of this Agreement. Except as required by law or governmental regulation, neither party shall disclose, advertise, or publish the terms and conditions of this Agreement without the prior written consent of the other party, except that either party may disclose the terms of this Agreement to potential acquirers, referral partners involved in an applicable transaction, accountants, attorneys and Affiliates pursuant to the terms of a non-disclosure or confidentiality agreement. If Customer is using a third party provider to host a Product, then such provider may also receive, subject to a confidentiality obligation, information related to the terms of this Agreement or Customer’s usage of the applicable Product.

+ +

4.5 Disclosure Required by Law. Notwithstanding anything to the contrary herein, each party may disclose the other party's Confidential Information in order to comply with applicable law and/or an order from a court or other governmental body of competent jurisdiction, and, in connection with compliance with such an order only, if such party: (i) unless prohibited by law, gives the other party prior written notice to such disclosure if the time between that order and such disclosure reasonably permits or, if time does not permit, gives the other party written notice of such disclosure promptly after complying with that order and (ii) fully cooperates with the other party, at the other party's cost and expense, in seeking a protective order, or confidential treatment, or taking other measures to oppose or limit such disclosure. Each party must not release any more of the other party's Confidential Information than is, in the opinion of its counsel, reasonably necessary to comply with an applicable order.

+ +

5. WARRANTIES AND DISCLAIMER OF WARRANTIES

+ +

5.1 Limited Support Services Performance Warranty. ClickHouse warrants that it will perform the Support Services in a professional, workmanlike manner, consistent with generally accepted industry practice, and in accordance with the Support Services Policy. In the event of a breach of the foregoing warranty, ClickHouse’s sole obligation, and Customer’s exclusive remedy, shall be for ClickHouse to re-perform the applicable Support Services.

+ +

5.2 Warranty Disclaimer. EXCEPT AS SET FORTH IN SECTION 5.1 ABOVE, THE SUPPORT SERVICES ARE PROVIDED “AS IS” WITHOUT WARRANTY OF ANY KIND AND CLICKHOUSE MAKES NO ADDITIONAL WARRANTIES, WHETHER EXPRESSED, IMPLIED OR STATUTORY, REGARDING OR RELATING TO THE SUPPORT SERVICES OR ANY MATERIALS FURNISHED OR PROVIDED TO CUSTOMER UNDER THIS AGREEMENT. TO THE MAXIMUM EXTENT PERMITTED UNDER APPLICABLE LAW, CLICKHOUSE SPECIFICALLY DISCLAIMS ALL IMPLIED WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NON-INFRINGEMENT WITH RESPECT TO THE SUPPORT SERVICES AND ANY MATERIALS FURNISHED OR PROVIDED TO CUSTOMER UNDER THIS AGREEMENT. CUSTOMER UNDERSTANDS AND AGREES THAT THE SUPPORT SERVICES AND ANY MATERIALS FURNISHED OR PROVIDED TO CUSTOMER UNDER THIS AGREEMENT ARE NOT DESIGNED OR INTENDED FOR USE IN THE OPERATION OF NUCLEAR FACILITIES, AIRCRAFT, WEAPONS SYSTEMS, OR LIFE SUPPORT SYSTEMS.

+ +

6. LIMITATION OF LIABILITY

+ +

6.1 Excluded Damages. IN NO EVENT SHALL CUSTOMER OR CLICKHOUSE, OR THEIR RESPECTIVE AFFILIATES, BE LIABLE FOR ANY LOSS OF PROFITS, LOSS OF USE, BUSINESS INTERRUPTION, LOSS OF DATA, COST OF SUBSTITUTE GOODS OR SERVICES, OR FOR ANY PUNITIVE, INDIRECT, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES OF ANY KIND IN CONNECTION WITH OR ARISING OUT OF THE PERFORMANCE OF OR FAILURE TO PERFORM THIS AGREEMENT, WHETHER ALLEGED AS A BREACH OF CONTRACT OR TORTIOUS CONDUCT, INCLUDING NEGLIGENCE, EVEN IF A PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.

+ +

6.2 Damages Cap. EXCEPT WITH RESPECT TO (I) A PARTY’S BREACH OF ITS OBLIGATIONS UNDER SECTION 4, (II) AMOUNTS PAYABLE BY CUSTOMER UNDER SECTION 3 OF THIS AGREEMENT AND EACH ORDER FORM, AND (III) CUSTOMER'S VIOLATIONS OF THE USE RESTRICTIONS SET FORTH IN THIS AGREEMENT, IN NO EVENT SHALL CLICKHOUSE'S OR CUSTOMER’S TOTAL, CUMULATIVE LIABILITY UNDER ANY ORDER FORM EXCEED THE AMOUNT PAID OR PAYABLE BY CUSTOMER TO CLICKHOUSE UNDER THIS AGREEMENT FOR THE AFFECTED SUPPORT SERVICES DELIVERED AND/OR MADE AVAILABLE TO CUSTOMER UNDER SUCH ORDER FORM FOR THE TWELVE (12) MONTH PERIOD IMMEDIATELY PRIOR TO THE FIRST EVENT GIVING RISE TO LIABILITY.

+ +

6.3 Basis of the Bargain. THE ALLOCATIONS OF LIABILITY IN THIS SECTION 6 REPRESENT THE AGREED AND BARGAINED FOR UNDERSTANDING OF THE PARTIES, AND THE COMPENSATION OF CLICKHOUSE FOR THE SUPPORT SERVICES PROVIDED HEREUNDER REFLECTS SUCH ALLOCATIONS. THE FOREGOING LIMITATIONS, EXCLUSIONS AND DISCLAIMERS WILL APPLY TO THE MAXIMUM EXTENT PERMITTED BY APPLICABLE LAW, EVEN IF ANY REMEDY FAILS IN ITS ESSENTIAL PURPOSE.

+ +

7. TERM AND TERMINATION

+ +

7.1 Subscription Term. The initial Subscription Term for each Subscription will commence and expire in accordance with the start date and end date set forth on the applicable Order Form, unless earlier terminated in in accordance with Section 7.3 below. Thereafter, each Subscription may be renewed for additional one (1) year periods upon the mutual written agreement of the parties. The initial Subscription Term, plus any subsequent renewal Subscription Term shall be the "Subscription Term".

+ +

7.2 Agreement Term. This Agreement will commence on the Effective Date and, unless earlier terminated in accordance with Section 7.3(b) below, continue in force and effect for a period of two (2) years. Thereafter, the term of this Agreement shall automatically renew for additional one (1) year periods unless either party gives written notice to the other of its intention not to renew the Agreement at least thirty (30) days prior to the expiration of the then-current term. The initial term of this Agreement, plus any subsequent renewal term, shall be the "Term" of this Agreement. Notwithstanding any expiration of this Agreement, its terms will continue to apply to any Subscription that has not been terminated or for which the Subscription Term has not expired.

+ +

7.3 Termination.

+
    +
  1. Subscriptions. Each party may terminate a Subscription upon giving notice in writing to the other party if the non-terminating party commits a material breach of this Agreement with respect to such Subscription, and has failed to cure such breach within thirty (30) days following a request in writing from the notifying party to do so. Upon the termination or expiration of a Subscription, the rights and obligations of the parties with respect thereto will, subject to Section 7.4 below, cease, provided that termination of a Subscription under this subsection (a) will not result in termination of any other Subscriptions.
  2. +
  3. Agreement. Either party may terminate this Agreement upon giving notice in writing to the other party if the non-terminating party commits a material breach of this Agreement with respect to any active Subscriptions hereunder, and has failed to cure such breach within thirty (30) days following a request in writing from the notifying party to do so. For the avoidance of doubt, termination of this Agreement under this subsection (b) will result in the termination of all Subscriptions and Order Forms.
  4. +
+ +

7.4 Survival. Upon the expiration or termination of an Order Form or this Agreement, (i) Customer shall have no further rights under any affected Subscription(s); and (ii) any payment obligations accrued under Section 3, as well as the provisions of Sections 1, 4, 5, 6, 7, 7.4 and 9 of this Agreement will survive such expiration or termination.

+ +

8. GENERAL

+ +

8.1 Anti-Corruption. Each party acknowledges that it is aware of, understands and has complied and will comply with, all applicable U.S. and foreign anti-corruption laws, including without limitation, the U.S. Foreign Corrupt Practices Act of 1977 and the U.K. Bribery Act of 2010, and similarly applicable anti-corruption and anti-bribery laws ("Anti-Corruption Laws"). Each party agrees that no one acting on its behalf will give, offer, agree or promise to give, or authorize the giving directly or indirectly, of any money or other thing of value, including travel, entertainment, or gifts, to anyone as an unlawful inducement or reward for favorable action or forbearance from action or the exercise of unlawful influence (a) to any governmental official or employee (including employees of government-owned and government-controlled corporations or agencies or public international organizations), (b) to any political party, official of a political party, or candidate, (c) to an intermediary for payment to any of the foregoing, or (d) to any other person or entity in a corrupt or improper effort to obtain or retain business or any commercial advantage, such as receiving a permit or license, or directing business to any person. Improper payments, provisions, bribes, kickbacks, influence payments, or other unlawful provisions to any person are prohibited under this Agreement.

+ +

8.2 Assignment. Neither party may assign this Agreement, in whole or in part, without the prior written consent of the other party, provided that no such consent will be required to assign this Agreement in its entirety to (i) an Affiliate that is able to satisfy the obligations of the assigning party under this Agreement or (ii) a successor in interest in connection with a merger, acquisition or sale of all or substantially all of the assigning party's assets. Any assignment in violation of this Section shall be void, ab initio, and of no effect. Subject to the foregoing, this Agreement is binding upon, inures to the benefit of and is enforceable by, the parties and their respective permitted successors and assigns.

+ +

8.3 Attorneys' Fees. If any action or proceeding, whether regulatory, administrative, at law or in equity is commenced or instituted to enforce or interpret any of the terms or provisions of this Agreement, the prevailing party in any such action or proceeding shall be entitled to recover its reasonable attorneys' fees, expert witness fees, costs of suit and expenses, in addition to any other relief to which such prevailing party may be entitled. As used herein, "prevailing party" includes without limitation, a party who dismisses an action for recovery hereunder in exchange for payment of the sums allegedly due, performance of covenants allegedly breached, or consideration substantially equal to the relief sought in the action.

+ +

8.4 California Consumer Privacy Act (CCPA). ClickHouse is a “Service Provider” as such term is defined under §1798.140(v) of the CCPA. As such ClickHouse shall not retain, use or disclose any personal information (as defined in the CCPA) received from Customer during the Term of this Agreement for any purpose other than the specific purpose of providing the Support Services specified in this Agreement or for such other business purpose as is specified in this Agreement.

+ +

8.5 Customer Identification. ClickHouse may identify Customer as a user of the Support Services, on its website, through a press release issued by ClickHouse and in other promotional materials.

+ +

8.6 Feedback. Customer, Customer’s Affiliates, and their respective agents, may volunteer feedback to ClickHouse, and/or its Affiliates, about the Support Services (“Feedback”). ClickHouse and its Affiliates shall be irrevocably entitled to use that Feedback, for any purpose and without any duty to account. provided that, in doing so, they may not breach their obligations of confidentiality under Section 4 of this Agreement.

+ +

8.7 Force Majeure. Except with respect to payment obligations, neither party will be liable for, or be considered to be in breach of, or in default under, this Agreement, as a result of any cause or condition beyond such party's reasonable control.

+ +

8.8 Governing Law, Jurisdiction and Venue.

+ +
    +
  1. Customers in California. If Customer is located in California (as determined by the Customer address on the applicable Order Form), this Agreement will be governed by the laws of the State of California, without regard to its conflict of laws principles, and all suits hereunder will be brought solely in Federal Court for the Northern District of California, or if that court lacks subject matter jurisdiction, in any California State Court located in Santa Clara County.
  2. +
  3. Customers Outside of California. If Customer is located anywhere other than California (as determined by the Customer address on the applicable Order Form), this Agreement will be governed by the laws of the State of Delaware, without regard to its conflict of laws principles, and all suits hereunder will be brought solely in Federal Court for the District of Delaware, or if that court lacks subject matter jurisdiction, in any Delaware State Court located in Wilmington, Delaware.
  4. + +
  5. All Customers. This Agreement shall not be governed by the 1980 UN Convention on Contracts for the International Sale of Goods. The parties hereby irrevocably waive any and all claims and defenses either might otherwise have in any action or proceeding in any of the applicable courts set forth in (a) or (b) above, based upon any alleged lack of personal jurisdiction, improper venue, forum non conveniens, or any similar claim or defense.
  6. + +
  7. Equitable Relief. A breach or threatened breach, by either party of Section 4 may cause irreparable harm for which damages at law may not provide adequate relief, and therefore the non-breaching party shall be entitled to seek injunctive relief without being required to post a bond.
  8. + +
+ +

8.9 Non-waiver. Any failure of either party to insist upon or enforce performance by the other party of any of the provisions of this Agreement or to exercise any rights or remedies under this Agreement will not be interpreted or construed as a waiver or relinquishment of such party's right to assert or rely upon such provision, right or remedy in that or any other instance.

+ +

8.10 Notices. Any notice or other communication under this Agreement given by either party to the other will be deemed to be properly given if given in writing and delivered in person or by e-mail, if acknowledged received by return e-mail or followed within one day by a delivered or mailed copy of such notice, or if mailed, properly addressed and stamped with the required postage, to the intended recipient at its address specified on an Order Form. Notices to ClickHouse may also be sent to legal@ClickHouse.com. Either party may from time to time change its address for notices under this Section by giving the other party notice of the change in accordance with this Section.

+ +

8.11 Relationship of the Parties. The relationship of the parties hereunder shall be that of independent contractors, and nothing herein shall be deemed or construed to create any employment, agency or fiduciary relationship between the parties. Each party shall be solely responsible for the supervision, direction, control and payment of its personnel, including, without limitation, for taxes, deductions and withholdings, compensation and benefits, and nothing herein will be deemed to result in either party having an employer-employee relationship with the personnel of the other party.

+ +

8.12 Severability. If any provision of this Agreement is held to be invalid or unenforceable, the remaining portions will remain in full force and effect and such provision will be enforced to the maximum extent possible so as to give effect the intent of the parties and will be reformed to the extent necessary to make such provision valid and enforceable.

+ +

8.13 Entire Agreement; Amendment. This Agreement, together with any Order Forms executed by the parties, and the Support Services Policy, each of which is hereby incorporated herein by this reference, constitutes the entire agreement between the parties concerning the subject matter hereof, and it supersedes, and its terms govern, all prior proposals, agreements, or other communications between the parties, oral or written, regarding such subject matter. This Agreement may be executed in any number of counterparts, each of which when so executed and delivered shall be deemed an original, and all of which together shall constitute one and the same agreement. Execution of a scanned copy will have the same force and effect as execution of an original, and a scanned signature will be deemed an original and valid signature. In the event of any conflict between the terms and conditions of any of the foregoing documents, the conflict shall be resolved based on the following order of precedence: (i) an applicable Order Form (but only for the transaction thereunder), (ii) an applicable Addendum (including any exhibits, attachments and addenda thereto), (iii) this Agreement, and (iv) the Support Services Policy. For the avoidance of doubt, the parties hereby expressly acknowledge and agree that if Customer issues any purchase orders or similar documents in connection with its purchase of a Subscription, it shall do so only for the purpose of Section 2.2(2) or for its own internal, administrative purposes and not with the intent to provide any contractual terms. By entering into this Agreement, whether prior to or following receipt of Customer's purchase order or any similar document, the parties are hereby expressly showing their intention not to be contractually bound by the contents of any such purchase order or similar document, which are hereby deemed rejected and extraneous to this Agreement, and ClickHouse's performance of this Agreement shall not amount to: (i) an acceptance by conduct of any terms set out or referred to in the purchase order or similar document; (ii) an amendment of this Agreement, nor (iii) an agreement to amend this Agreement. This Agreement shall not be modified except by a subsequently dated, written amendment that expressly amends this Agreement and which is signed on behalf of ClickHouse and Customer by their duly authorized representatives. The parties agree that the terms and conditions of this Agreement are a result of mutual negotiations. Therefore, the rule of construction that any ambiguity shall apply against the drafter is not applicable and will not apply to this Agreement. Any ambiguity shall be reasonably construed as to its fair meaning and not strictly for or against one party regardless of who authored the ambiguous language.

+ + +
+
\ No newline at end of file diff --git a/website/templates/support/agreement-hero.html b/website/templates/support/agreement-hero.html new file mode 100644 index 00000000000..ea97fb7729a --- /dev/null +++ b/website/templates/support/agreement-hero.html @@ -0,0 +1,10 @@ +
+
+
+ +

+ {{ _('Clickhouse, Inc.
Subscription Agreement') }} +

+ +
+
\ No newline at end of file From 78afe31397bdb451fa7adce604c68195b5746ccf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 11 Feb 2022 01:17:20 +0300 Subject: [PATCH 085/164] fix --- src/Interpreters/InterpreterCreateQuery.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ac415214892..5be320f267d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -780,7 +780,10 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const if (create.as_table_function) return; - if (create.is_dictionary || create.is_ordinary_view || create.is_live_view) + if (create.is_dictionary || create.is_ordinary_view || create.is_live_view || create.is_window_view) + return; + + if (create.is_materialized_view && create.to_table_id) return; if (create.temporary) From a9a2d76c910b1d6291e4ce677c3490735875e862 Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 11 Feb 2022 12:06:42 +0800 Subject: [PATCH 086/164] Translate zh/engines/database-engines/materialized-mysql: create new file --- .../materialized-mysql.md.new | 274 ++++++++++++++++++ 1 file changed, 274 insertions(+) create mode 100644 docs/zh/engines/database-engines/materialized-mysql.md.new diff --git a/docs/zh/engines/database-engines/materialized-mysql.md.new b/docs/zh/engines/database-engines/materialized-mysql.md.new new file mode 100644 index 00000000000..f654013494a --- /dev/null +++ b/docs/zh/engines/database-engines/materialized-mysql.md.new @@ -0,0 +1,274 @@ +--- +toc_priority: 29 +toc_title: MaterializedMySQL +--- + +# [experimental] MaterializedMySQL {#materialized-mysql} + +!!! warning "警告" + 这是一个实验性的特性,不应该在生产中使用. + + +创建ClickHouse数据库,包含MySQL中所有的表,以及这些表中的所有数据。 + +ClickHouse服务器作为MySQL副本工作。它读取binlog并执行DDL和DML查询。 + +## 创建数据库 {#creating-a-database} + +``` sql +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] +ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'password') [SETTINGS ...] +[TABLE OVERRIDE table1 (...), TABLE OVERRIDE table2 (...)] +``` + +**引擎参数** + +- `host:port` — MySQL 服务地址. +- `database` — MySQL 数据库名称. +- `user` — MySQL 用户名. +- `password` — MySQL 用户密码. + +**引擎配置** + + +- `max_rows_in_buffer` — 允许在内存中缓存数据的最大行数(对于单个表和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值:`65 505`。 +- `max_bytes_in_buffer` - 允许在内存中缓存数据的最大字节数(对于单个表和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值: `1 048 576 `。 +- `max_rows_in_buffers` - 允许在内存中缓存数据的最大行数(用于数据库和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值: `65 505`。 +- `max_bytes_in_buffers` - 允许在内存中缓存数据的最大字节数(用于数据库和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值: `1 048 576`。 +- `max_flush_data_time ` - 允许数据在内存中缓存的最大毫秒数(对于数据库和无法查询的缓存数据)。当超过这个时间,数据将被物化。默认值: `1000`。 +- `max_wait_time_when_mysql_unavailable` - MySQL不可用时的重试间隔(毫秒)。负值禁用重试。默认值:`1000`。 +— `allows_query_when_mysql_lost `—允许在MySQL丢失时查询物化表。默认值:`0`(`false`)。 + +```sql +CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') + SETTINGS + allows_query_when_mysql_lost=true, + max_wait_time_when_mysql_unavailable=10000; +``` + +**MySQL服务器端配置** + +为了`MaterializedMySQL`的正确工作,有一些必须设置的`MySQL`端配置设置: + +- `default_authentication_plugin = mysql_native_password `,因为 `MaterializedMySQL` 只能授权使用该方法。 +- `gtid_mode = on`,因为基于GTID的日志记录是提供正确的 `MaterializedMySQL`复制的强制要求。 + +!!! attention "注意" + 当打开`gtid_mode`时,您还应该指定`enforce_gtid_consistency = on`。 + +## 虚拟列 {#virtual-columns} + +当使用`MaterializeMySQL`数据库引擎时,[ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md)表与虚拟的`_sign`和`_version`列一起使用。 + +- `_version` — 事务版本. 类型 [UInt64](../../sql-reference/data-types/int-uint.md). +- `_sign` — 删除标记. 类型 [Int8](../../sql-reference/data-types/int-uint.md). 可能的值: + - `1` — 行没有删除, + - `-1` — 行已被删除. + +## 支持的数据类型 {#data_types-support} + +| MySQL | ClickHouse | +|-------------------------|--------------------------------------------------------------| +| TINY | [Int8](../../sql-reference/data-types/int-uint.md) | +| SHORT | [Int16](../../sql-reference/data-types/int-uint.md) | +| INT24 | [Int32](../../sql-reference/data-types/int-uint.md) | +| LONG | [UInt32](../../sql-reference/data-types/int-uint.md) | +| LONGLONG | [UInt64](../../sql-reference/data-types/int-uint.md) | +| FLOAT | [Float32](../../sql-reference/data-types/float.md) | +| DOUBLE | [Float64](../../sql-reference/data-types/float.md) | +| DECIMAL, NEWDECIMAL | [Decimal](../../sql-reference/data-types/decimal.md) | +| DATE, NEWDATE | [Date](../../sql-reference/data-types/date.md) | +| DATETIME, TIMESTAMP | [DateTime](../../sql-reference/data-types/datetime.md) | +| DATETIME2, TIMESTAMP2 | [DateTime64](../../sql-reference/data-types/datetime64.md) | +| YEAR | [UInt16](../../sql-reference/data-types/int-uint.md) | +| TIME | [Int64](../../sql-reference/data-types/int-uint.md) | +| ENUM | [Enum](../../sql-reference/data-types/enum.md) | +| STRING | [String](../../sql-reference/data-types/string.md) | +| VARCHAR, VAR_STRING | [String](../../sql-reference/data-types/string.md) | +| BLOB | [String](../../sql-reference/data-types/string.md) | +| GEOMETRY | [String](../../sql-reference/data-types/string.md) | +| BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) | +| BIT | [UInt64](../../sql-reference/data-types/int-uint.md) | +| SET | [UInt64](../../sql-reference/data-types/int-uint.md) | + +[Nullable](../../sql-reference/data-types/nullable.md) 已经被支持. + +MySQL中的Time 类型,会被ClickHouse转换成微秒来存储 + +不支持其他类型。如果MySQL表包含此类类型的列,ClickHouse抛出异常"Unhandled data type"并停止复制。 + +## 规范和推荐用法 {#specifics-and-recommendations} + +### 兼容性限制 {#compatibility-restrictions} + +除了数据类型的限制之外,还有一些限制与`MySQL`数据库相比有所不同,这应该在复制之前解决: + +- `MySQL` 中的每个表都应该包含 `PRIMARY KEY`。 +- 对于表的复制,那些包含 `ENUM` 字段值超出范围的行(在 `ENUM` 签名中指定)将不起作用。 + +### DDL Queries {#ddl-queries} + +MySQL DDL 语句会被转换成对应的ClickHouse DDL 语句,比如: ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). 如果ClickHouse 无法解析某些语句DDL 操作,则会跳过。 + + +### 数据复制 {#data-replication} + +MaterializedMySQL不支持直接的 `INSERT`, `DELETE` 和 `UPDATE` 查询。然而,它们在数据复制方面得到了支持: + +- MySQL `INSERT`查询被转换为`_sign=1`的INSERT查询。 +- MySQL `DELETE`查询被转换为`INSERT`,并且`_sign=-1`。 +- 如果主键被修改了,MySQL的 `UPDATE` 查询将被转换为 `INSERT` 带 `_sign=1` 和INSERT 带有_sign=-1;如果主键没有被修改,则转换为`INSERT`和`_sign=1`。 + +### MaterializedMySQL 数据表查询 {#select} + +`SELECT` 查询从 `MaterializedMySQL`表有一些细节: + + - 如果在SELECT查询中没有指定`_version`,则 [FINAL](../../sql-reference/statements/select/from.md#select-from- FINAL)修饰符被使用,所以只有带有 `MAX(_version)`的行会返回每个主键值。 + + - 如果在SELECT查询中没有指定 `_sign`,则默认使用 `WHERE _sign=1 `。所以被删除的行不是 +包含在结果集中。 + + - 结果包括列注释,以防MySQL数据库表中存在这些列注释。 + +### 索引转换 {#index-conversion} + +在ClickHouse表中,MySQL的 `PRIMARY KEY` 和 `INDEX` 子句被转换为 `ORDER BY` 元组。 + +ClickHouse只有一个物理排序,由 `order by` 条件决定。要创建一个新的物理排序,请使用[materialized views](../../sql-reference/statements/create/view.md#materialized)。 + +**注意** + +- `_sign=-1` 的行不会被物理地从表中删除。 +- 级联 `UPDATE/DELETE` 查询不支持 `MaterializedMySQL` 引擎,因为他们在 MySQL binlog中不可见的 +— 复制很容易被破坏。 +— 禁止对数据库和表进行手工操作。 +- `MaterializedMySQL` 受[optimize_on_insert](../../operations/settings/settings.md#optimize-on-insert)设置的影响。当MySQL服务器中的一个表发生变化时,数据会合并到 `MaterializedMySQL` 数据库中相应的表中。 + +### 表重写 {#table-overrides} + +表覆盖可用于自定义ClickHouse DDL查询,从而允许您对应用程序进行模式优化。这对于控制分区特别有用,分区对MaterializedMySQL的整体性能非常重要。 + +这些是你可以对MaterializedMySQL表重写的模式转换操作: + + * 修改列类型。必须与原始类型兼容,否则复制将失败。例如,可以将`UInt32`列修改为`UInt64`,不能将 `String` 列修改为 `Array(String)`。 + * 修改 [column TTL](../table-engines/mergetree-family/mergetree/#mergetree-column-ttl). + * 修改 [column compression codec](../../sql-reference/statements/create/table/#codecs). + * 增加 [ALIAS columns](../../sql-reference/statements/create/table/#alias). + * 增加 [skipping indexes](../table-engines/mergetree-family/mergetree/#table_engine-mergetree-data_skipping-indexes) + * 增加 [projections](../table-engines/mergetree-family/mergetree/#projections). + 请注意,当使用 `SELECT ... FINAL ` (MaterializedMySQL默认是这样做的) 时,预测优化是被禁用的,所以这里是受限的, `INDEX ... TYPE hypothesis `[在v21.12的博客文章中描述]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/)可能在这种情况下更有用。 + * 修改 [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key/) + * 修改 [ORDER BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * 修改 [PRIMARY KEY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * 增加 [SAMPLE BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * 增加 [table TTL](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + +```sql +CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) +[SETTINGS ...] +[TABLE OVERRIDE table_name ( + [COLUMNS ( + [col_name [datatype] [ALIAS expr] [CODEC(...)] [TTL expr], ...] + [INDEX index_name expr TYPE indextype[(...)] GRANULARITY val, ...] + [PROJECTION projection_name (SELECT [GROUP BY] [ORDER BY]), ...] + )] + [ORDER BY expr] + [PRIMARY KEY expr] + [PARTITION BY expr] + [SAMPLE BY expr] + [TTL expr] +), ...] +``` + +示例: + +```sql +CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) +TABLE OVERRIDE table1 ( + COLUMNS ( + userid UUID, + category LowCardinality(String), + timestamp DateTime CODEC(Delta, Default) + ) + PARTITION BY toYear(timestamp) +), +TABLE OVERRIDE table2 ( + COLUMNS ( + client_ip String TTL created + INTERVAL 72 HOUR + ) + SAMPLE BY ip_hash +) +``` + + +`COLUMNS`列表是稀疏的;根据指定修改现有列,添加额外的ALIAS列。不可能添加普通列或实体化列。具有不同类型的已修改列必须可从原始类型赋值。在执行`CREATE DATABASE` 查询时,目前还没有验证这个或类似的问题,因此需要格外小心。 + +您可以为还不存在的表指定重写。 + +!!! warning "警告" + 如果使用时不小心,很容易用表重写中断复制。例如: + + * 如果一个ALIAS列被添加了一个表覆盖,并且一个具有相同名称的列后来被添加到源MySQL表,在ClickHouse中转换后的ALTER table查询将失败并停止复制。 + * 目前可以添加引用可空列的覆盖,而非空列是必需的,例如 `ORDER BY` 或 `PARTITION BY`。这将导致CREATE TABLE查询失败,也会导致复制停止。 + +## 使用示例 {#examples-of-use} + + MySQL 查询语句: + +``` sql +mysql> CREATE DATABASE db; +mysql> CREATE TABLE db.test (a INT PRIMARY KEY, b INT); +mysql> INSERT INTO db.test VALUES (1, 11), (2, 22); +mysql> DELETE FROM db.test WHERE a=1; +mysql> ALTER TABLE db.test ADD COLUMN c VARCHAR(16); +mysql> UPDATE db.test SET c='Wow!', b=222; +mysql> SELECT * FROM test; +``` + +```text +┌─a─┬───b─┬─c────┐ +│ 2 │ 222 │ Wow! │ +└───┴─────┴──────┘ +``` + +ClickHouse中的数据库,与MySQL服务器交换数据: + +创建的数据库和表: + +``` sql +CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***'); +SHOW TABLES FROM mysql; +``` + +``` text +┌─name─┐ +│ test │ +└──────┘ +``` + +数据插入之后: + +``` sql +SELECT * FROM mysql.test; +``` + +``` text +┌─a─┬──b─┐ +│ 1 │ 11 │ +│ 2 │ 22 │ +└───┴────┘ +``` + +删除数据后,添加列并更新: + +``` sql +SELECT * FROM mysql.test; +``` + +``` text +┌─a─┬───b─┬─c────┐ +│ 2 │ 222 │ Wow! │ +└───┴─────┴──────┘ +``` + +[来源文章](https://clickhouse.com/docs/en/engines/database-engines/materialized-mysql/) From 3f622e82f74688c7d26b94d7503d7b8809b127de Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 11 Feb 2022 12:08:01 +0800 Subject: [PATCH 087/164] Translate zh/engines/database-engines/materialized-mysql: revert en doc --- .../database-engines/materialized-mysql.md | 202 ++++++++++-------- 1 file changed, 109 insertions(+), 93 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index f654013494a..bcb026aa0dc 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -5,15 +5,14 @@ toc_title: MaterializedMySQL # [experimental] MaterializedMySQL {#materialized-mysql} -!!! warning "警告" - 这是一个实验性的特性,不应该在生产中使用. +!!! warning "Warning" + This is an experimental feature that should not be used in production. +Creates ClickHouse database with all the tables existing in MySQL, and all the data in those tables. -创建ClickHouse数据库,包含MySQL中所有的表,以及这些表中的所有数据。 +ClickHouse server works as MySQL replica. It reads binlog and performs DDL and DML queries. -ClickHouse服务器作为MySQL副本工作。它读取binlog并执行DDL和DML查询。 - -## 创建数据库 {#creating-a-database} +## Creating a Database {#creating-a-database} ``` sql CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] @@ -21,23 +20,22 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo [TABLE OVERRIDE table1 (...), TABLE OVERRIDE table2 (...)] ``` -**引擎参数** +**Engine Parameters** -- `host:port` — MySQL 服务地址. -- `database` — MySQL 数据库名称. -- `user` — MySQL 用户名. -- `password` — MySQL 用户密码. +- `host:port` — MySQL server endpoint. +- `database` — MySQL database name. +- `user` — MySQL user. +- `password` — User password. -**引擎配置** +**Engine Settings** - -- `max_rows_in_buffer` — 允许在内存中缓存数据的最大行数(对于单个表和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值:`65 505`。 -- `max_bytes_in_buffer` - 允许在内存中缓存数据的最大字节数(对于单个表和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值: `1 048 576 `。 -- `max_rows_in_buffers` - 允许在内存中缓存数据的最大行数(用于数据库和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值: `65 505`。 -- `max_bytes_in_buffers` - 允许在内存中缓存数据的最大字节数(用于数据库和无法查询的缓存数据)。当超过这个数字时,数据将被物化。默认值: `1 048 576`。 -- `max_flush_data_time ` - 允许数据在内存中缓存的最大毫秒数(对于数据库和无法查询的缓存数据)。当超过这个时间,数据将被物化。默认值: `1000`。 -- `max_wait_time_when_mysql_unavailable` - MySQL不可用时的重试间隔(毫秒)。负值禁用重试。默认值:`1000`。 -— `allows_query_when_mysql_lost `—允许在MySQL丢失时查询物化表。默认值:`0`(`false`)。 +- `max_rows_in_buffer` — Maximum number of rows that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. +- `max_bytes_in_buffer` — Maximum number of bytes that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_rows_in_buffers` — Maximum number of rows that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`. +- `max_bytes_in_buffers` — Maximum number of bytes that data is allowed to cache in memory (for database and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`. +- `max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`. +- `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disables retry. Default: `1000`. +- `allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`). ```sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***') @@ -46,26 +44,26 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', max_wait_time_when_mysql_unavailable=10000; ``` -**MySQL服务器端配置** +**Settings on MySQL-server Side** -为了`MaterializedMySQL`的正确工作,有一些必须设置的`MySQL`端配置设置: +For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set: -- `default_authentication_plugin = mysql_native_password `,因为 `MaterializedMySQL` 只能授权使用该方法。 -- `gtid_mode = on`,因为基于GTID的日志记录是提供正确的 `MaterializedMySQL`复制的强制要求。 +- `default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method. +- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializedMySQL` replication. -!!! attention "注意" - 当打开`gtid_mode`时,您还应该指定`enforce_gtid_consistency = on`。 +!!! attention "Attention" + While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = on`. -## 虚拟列 {#virtual-columns} +## Virtual Columns {#virtual-columns} -当使用`MaterializeMySQL`数据库引擎时,[ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md)表与虚拟的`_sign`和`_version`列一起使用。 +When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. -- `_version` — 事务版本. 类型 [UInt64](../../sql-reference/data-types/int-uint.md). -- `_sign` — 删除标记. 类型 [Int8](../../sql-reference/data-types/int-uint.md). 可能的值: - - `1` — 行没有删除, - - `-1` — 行已被删除. +- `_version` — Transaction counter. Type [UInt64](../../sql-reference/data-types/int-uint.md). +- `_sign` — Deletion mark. Type [Int8](../../sql-reference/data-types/int-uint.md). Possible values: + - `1` — Row is not deleted, + - `-1` — Row is deleted. -## 支持的数据类型 {#data_types-support} +## Data Types Support {#data_types-support} | MySQL | ClickHouse | |-------------------------|--------------------------------------------------------------| @@ -91,77 +89,91 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', | BIT | [UInt64](../../sql-reference/data-types/int-uint.md) | | SET | [UInt64](../../sql-reference/data-types/int-uint.md) | -[Nullable](../../sql-reference/data-types/nullable.md) 已经被支持. +[Nullable](../../sql-reference/data-types/nullable.md) is supported. -MySQL中的Time 类型,会被ClickHouse转换成微秒来存储 +The data of TIME type in MySQL is converted to microseconds in ClickHouse. -不支持其他类型。如果MySQL表包含此类类型的列,ClickHouse抛出异常"Unhandled data type"并停止复制。 +Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication. -## 规范和推荐用法 {#specifics-and-recommendations} +## Specifics and Recommendations {#specifics-and-recommendations} -### 兼容性限制 {#compatibility-restrictions} +### Compatibility Restrictions {#compatibility-restrictions} -除了数据类型的限制之外,还有一些限制与`MySQL`数据库相比有所不同,这应该在复制之前解决: +Apart of the data types limitations there are few restrictions comparing to `MySQL` databases, that should be resolved before replication will be possible: -- `MySQL` 中的每个表都应该包含 `PRIMARY KEY`。 -- 对于表的复制,那些包含 `ENUM` 字段值超出范围的行(在 `ENUM` 签名中指定)将不起作用。 +- Each table in `MySQL` should contain `PRIMARY KEY`. + +- Replication for tables, those are containing rows with `ENUM` field values out of range (specified in `ENUM` signature) will not work. ### DDL Queries {#ddl-queries} -MySQL DDL 语句会被转换成对应的ClickHouse DDL 语句,比如: ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). 如果ClickHouse 无法解析某些语句DDL 操作,则会跳过。 +MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop.md), [RENAME](../../sql-reference/statements/rename.md)). If ClickHouse cannot parse some DDL query, the query is ignored. +### Data Replication {#data-replication} -### 数据复制 {#data-replication} +`MaterializedMySQL` does not support direct `INSERT`, `DELETE` and `UPDATE` queries. However, they are supported in terms of data replication: -MaterializedMySQL不支持直接的 `INSERT`, `DELETE` 和 `UPDATE` 查询。然而,它们在数据复制方面得到了支持: +- MySQL `INSERT` query is converted into `INSERT` with `_sign=1`. -- MySQL `INSERT`查询被转换为`_sign=1`的INSERT查询。 -- MySQL `DELETE`查询被转换为`INSERT`,并且`_sign=-1`。 -- 如果主键被修改了,MySQL的 `UPDATE` 查询将被转换为 `INSERT` 带 `_sign=1` 和INSERT 带有_sign=-1;如果主键没有被修改,则转换为`INSERT`和`_sign=1`。 +- MySQL `DELETE` query is converted into `INSERT` with `_sign=-1`. -### MaterializedMySQL 数据表查询 {#select} +- MySQL `UPDATE` query is converted into `INSERT` with `_sign=-1` and `INSERT` with `_sign=1` if the primary key has been changed, or + `INSERT` with `_sign=1` if not. -`SELECT` 查询从 `MaterializedMySQL`表有一些细节: +### Selecting from MaterializedMySQL Tables {#select} - - 如果在SELECT查询中没有指定`_version`,则 [FINAL](../../sql-reference/statements/select/from.md#select-from- FINAL)修饰符被使用,所以只有带有 `MAX(_version)`的行会返回每个主键值。 +`SELECT` query from `MaterializedMySQL` tables has some specifics: - - 如果在SELECT查询中没有指定 `_sign`,则默认使用 `WHERE _sign=1 `。所以被删除的行不是 -包含在结果集中。 +- If `_version` is not specified in the `SELECT` query, the + [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier is used, so only rows with + `MAX(_version)` are returned for each primary key value. - - 结果包括列注释,以防MySQL数据库表中存在这些列注释。 +- If `_sign` is not specified in the `SELECT` query, `WHERE _sign=1` is used by default. So the deleted rows are not + included into the result set. -### 索引转换 {#index-conversion} +- The result includes columns comments in case they exist in MySQL database tables. -在ClickHouse表中,MySQL的 `PRIMARY KEY` 和 `INDEX` 子句被转换为 `ORDER BY` 元组。 +### Index Conversion {#index-conversion} -ClickHouse只有一个物理排序,由 `order by` 条件决定。要创建一个新的物理排序,请使用[materialized views](../../sql-reference/statements/create/view.md#materialized)。 +MySQL `PRIMARY KEY` and `INDEX` clauses are converted into `ORDER BY` tuples in ClickHouse tables. -**注意** +ClickHouse has only one physical order, which is determined by `ORDER BY` clause. To create a new physical order, use +[materialized views](../../sql-reference/statements/create/view.md#materialized). -- `_sign=-1` 的行不会被物理地从表中删除。 -- 级联 `UPDATE/DELETE` 查询不支持 `MaterializedMySQL` 引擎,因为他们在 MySQL binlog中不可见的 -— 复制很容易被破坏。 -— 禁止对数据库和表进行手工操作。 -- `MaterializedMySQL` 受[optimize_on_insert](../../operations/settings/settings.md#optimize-on-insert)设置的影响。当MySQL服务器中的一个表发生变化时,数据会合并到 `MaterializedMySQL` 数据库中相应的表中。 +**Notes** -### 表重写 {#table-overrides} +- Rows with `_sign=-1` are not deleted physically from the tables. +- Cascade `UPDATE/DELETE` queries are not supported by the `MaterializedMySQL` engine, as they are not visible in the + MySQL binlog. +- Replication can be easily broken. +- Manual operations on database and tables are forbidden. +- `MaterializedMySQL` is affected by the [optimize_on_insert](../../operations/settings/settings.md#optimize-on-insert) + setting. Data is merged in the corresponding table in the `MaterializedMySQL` database when a table in the MySQL + server changes. -表覆盖可用于自定义ClickHouse DDL查询,从而允许您对应用程序进行模式优化。这对于控制分区特别有用,分区对MaterializedMySQL的整体性能非常重要。 +### Table Overrides {#table-overrides} -这些是你可以对MaterializedMySQL表重写的模式转换操作: +Table overrides can be used to customize the ClickHouse DDL queries, allowing you to make schema optimizations for your +application. This is especially useful for controlling partitioning, which is important for the overall performance of +MaterializedMySQL. - * 修改列类型。必须与原始类型兼容,否则复制将失败。例如,可以将`UInt32`列修改为`UInt64`,不能将 `String` 列修改为 `Array(String)`。 - * 修改 [column TTL](../table-engines/mergetree-family/mergetree/#mergetree-column-ttl). - * 修改 [column compression codec](../../sql-reference/statements/create/table/#codecs). - * 增加 [ALIAS columns](../../sql-reference/statements/create/table/#alias). - * 增加 [skipping indexes](../table-engines/mergetree-family/mergetree/#table_engine-mergetree-data_skipping-indexes) - * 增加 [projections](../table-engines/mergetree-family/mergetree/#projections). - 请注意,当使用 `SELECT ... FINAL ` (MaterializedMySQL默认是这样做的) 时,预测优化是被禁用的,所以这里是受限的, `INDEX ... TYPE hypothesis `[在v21.12的博客文章中描述]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/)可能在这种情况下更有用。 - * 修改 [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key/) - * 修改 [ORDER BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * 修改 [PRIMARY KEY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * 增加 [SAMPLE BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * 增加 [table TTL](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) +These are the schema conversion manipulations you can do with table overrides for MaterializedMySQL: + + * Modify column type. Must be compatible with the original type, or replication will fail. For example, + you can modify a UInt32 column to UInt64, but you can not modify a String column to Array(String). + * Modify [column TTL](../table-engines/mergetree-family/mergetree/#mergetree-column-ttl). + * Modify [column compression codec](../../sql-reference/statements/create/table/#codecs). + * Add [ALIAS columns](../../sql-reference/statements/create/table/#alias). + * Add [skipping indexes](../table-engines/mergetree-family/mergetree/#table_engine-mergetree-data_skipping-indexes) + * Add [projections](../table-engines/mergetree-family/mergetree/#projections). Note that projection optimizations are + disabled when using `SELECT ... FINAL` (which MaterializedMySQL does by default), so their utility is limited here. + `INDEX ... TYPE hypothesis` as [described in the v21.12 blog post]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/) + may be more useful in this case. + * Modify [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key/) + * Modify [ORDER BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * Modify [PRIMARY KEY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * Add [SAMPLE BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * Add [table TTL](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) ```sql CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) @@ -180,7 +192,7 @@ CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) ), ...] ``` -示例: +Example: ```sql CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) @@ -200,20 +212,24 @@ TABLE OVERRIDE table2 ( ) ``` +The `COLUMNS` list is sparse; existing columns are modified as specified, extra ALIAS columns are added. It is not +possible to add ordinary or MATERIALIZED columns. Modified columns with a different type must be assignable from the +original type. There is currently no validation of this or similar issues when the `CREATE DATABASE` query executes, so +extra care needs to be taken. -`COLUMNS`列表是稀疏的;根据指定修改现有列,添加额外的ALIAS列。不可能添加普通列或实体化列。具有不同类型的已修改列必须可从原始类型赋值。在执行`CREATE DATABASE` 查询时,目前还没有验证这个或类似的问题,因此需要格外小心。 +You may specify overrides for tables that do not exist yet. -您可以为还不存在的表指定重写。 +!!! warning "Warning" + It is easy to break replication with table overrides if not used with care. For example: + + * If an ALIAS column is added with a table override, and a column with the same name is later added to the source + MySQL table, the converted ALTER TABLE query in ClickHouse will fail and replication stops. + * It is currently possible to add overrides that reference nullable columns where not-nullable are required, such as in + `ORDER BY` or `PARTITION BY`. This will cause CREATE TABLE queries that will fail, also causing replication to stop. -!!! warning "警告" - 如果使用时不小心,很容易用表重写中断复制。例如: +## Examples of Use {#examples-of-use} - * 如果一个ALIAS列被添加了一个表覆盖,并且一个具有相同名称的列后来被添加到源MySQL表,在ClickHouse中转换后的ALTER table查询将失败并停止复制。 - * 目前可以添加引用可空列的覆盖,而非空列是必需的,例如 `ORDER BY` 或 `PARTITION BY`。这将导致CREATE TABLE查询失败,也会导致复制停止。 - -## 使用示例 {#examples-of-use} - - MySQL 查询语句: +Queries in MySQL: ``` sql mysql> CREATE DATABASE db; @@ -231,9 +247,9 @@ mysql> SELECT * FROM test; └───┴─────┴──────┘ ``` -ClickHouse中的数据库,与MySQL服务器交换数据: +Database in ClickHouse, exchanging data with the MySQL server: -创建的数据库和表: +The database and the table created: ``` sql CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***'); @@ -246,7 +262,7 @@ SHOW TABLES FROM mysql; └──────┘ ``` -数据插入之后: +After inserting data: ``` sql SELECT * FROM mysql.test; @@ -259,7 +275,7 @@ SELECT * FROM mysql.test; └───┴────┘ ``` -删除数据后,添加列并更新: +After deleting data, adding the column and updating: ``` sql SELECT * FROM mysql.test; @@ -271,4 +287,4 @@ SELECT * FROM mysql.test; └───┴─────┴──────┘ ``` -[来源文章](https://clickhouse.com/docs/en/engines/database-engines/materialized-mysql/) +[Original article](https://clickhouse.com/docs/en/engines/database-engines/materialized-mysql/) From 84d216a3b308849ab92e65c6237b902ed720af26 Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 11 Feb 2022 12:09:02 +0800 Subject: [PATCH 088/164] Translate zh/engines/database-engines/materialized-mysql: rename link file --- .../{materialized-mysql.md => materialized-mysql.md.old} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/zh/engines/database-engines/{materialized-mysql.md => materialized-mysql.md.old} (100%) diff --git a/docs/zh/engines/database-engines/materialized-mysql.md b/docs/zh/engines/database-engines/materialized-mysql.md.old similarity index 100% rename from docs/zh/engines/database-engines/materialized-mysql.md rename to docs/zh/engines/database-engines/materialized-mysql.md.old From b455ef75363d8c3b5e49d85a1580845d5879a3b1 Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 11 Feb 2022 12:09:48 +0800 Subject: [PATCH 089/164] Translate zh/engines/database-engines/materialized-mysql: rename back to zh --- .../{materialized-mysql.md.new => materialized-mysql.md} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/zh/engines/database-engines/{materialized-mysql.md.new => materialized-mysql.md} (100%) diff --git a/docs/zh/engines/database-engines/materialized-mysql.md.new b/docs/zh/engines/database-engines/materialized-mysql.md similarity index 100% rename from docs/zh/engines/database-engines/materialized-mysql.md.new rename to docs/zh/engines/database-engines/materialized-mysql.md From 2da9d6b8b54a1faf3663e88aab6a69d03cabad5b Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 11 Feb 2022 12:10:28 +0800 Subject: [PATCH 090/164] Translate zh/engines/database-engines/materialized-mysql: remove old back file --- docs/zh/engines/database-engines/materialized-mysql.md.old | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/engines/database-engines/materialized-mysql.md.old diff --git a/docs/zh/engines/database-engines/materialized-mysql.md.old b/docs/zh/engines/database-engines/materialized-mysql.md.old deleted file mode 120000 index 02118b85df4..00000000000 --- a/docs/zh/engines/database-engines/materialized-mysql.md.old +++ /dev/null @@ -1 +0,0 @@ -../../../en/engines/database-engines/materialized-mysql.md \ No newline at end of file From e148b43a29e3bf5f8ad73160c9c8b234508174b7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 10 Feb 2022 16:39:33 +0700 Subject: [PATCH 091/164] Privileges CREATE/ALTER/DROP ROW POLICY now can be granted on a table or on database.* as well as globally *.* --- src/Access/Common/AccessType.h | 8 ++-- src/Access/tests/gtest_access_rights_ops.cpp | 10 +++- .../InterpreterCreateRowPolicyQuery.cpp | 24 ++++++++-- .../Access/InterpreterCreateRowPolicyQuery.h | 4 +- .../InterpreterDropAccessEntityQuery.cpp | 37 +++++++++++--- .../Access/InterpreterDropAccessEntityQuery.h | 1 - ...InterpreterShowCreateAccessEntityQuery.cpp | 48 ++++++++++++++++--- .../integration/test_grant_and_revoke/test.py | 7 ++- tests/integration/test_row_policy/test.py | 46 ++++++++++++++++++ .../01271_show_privileges.reference | 8 ++-- 10 files changed, 163 insertions(+), 30 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 16d3de40ec3..0b69bd5fd0e 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -113,9 +113,9 @@ enum class AccessType M(ALTER_ROLE, "", GLOBAL, ACCESS_MANAGEMENT) \ M(DROP_ROLE, "", GLOBAL, ACCESS_MANAGEMENT) \ M(ROLE_ADMIN, "", GLOBAL, ACCESS_MANAGEMENT) /* allows to grant and revoke the roles which are not granted to the current user with admin option */\ - M(CREATE_ROW_POLICY, "CREATE POLICY", GLOBAL, ACCESS_MANAGEMENT) \ - M(ALTER_ROW_POLICY, "ALTER POLICY", GLOBAL, ACCESS_MANAGEMENT) \ - M(DROP_ROW_POLICY, "DROP POLICY", GLOBAL, ACCESS_MANAGEMENT) \ + M(CREATE_ROW_POLICY, "CREATE POLICY", TABLE, ACCESS_MANAGEMENT) \ + M(ALTER_ROW_POLICY, "ALTER POLICY", TABLE, ACCESS_MANAGEMENT) \ + M(DROP_ROW_POLICY, "DROP POLICY", TABLE, ACCESS_MANAGEMENT) \ M(CREATE_QUOTA, "", GLOBAL, ACCESS_MANAGEMENT) \ M(ALTER_QUOTA, "", GLOBAL, ACCESS_MANAGEMENT) \ M(DROP_QUOTA, "", GLOBAL, ACCESS_MANAGEMENT) \ @@ -124,7 +124,7 @@ enum class AccessType M(DROP_SETTINGS_PROFILE, "DROP PROFILE", GLOBAL, ACCESS_MANAGEMENT) \ M(SHOW_USERS, "SHOW CREATE USER", GLOBAL, SHOW_ACCESS) \ M(SHOW_ROLES, "SHOW CREATE ROLE", GLOBAL, SHOW_ACCESS) \ - M(SHOW_ROW_POLICIES, "SHOW POLICIES, SHOW CREATE ROW POLICY, SHOW CREATE POLICY", GLOBAL, SHOW_ACCESS) \ + M(SHOW_ROW_POLICIES, "SHOW POLICIES, SHOW CREATE ROW POLICY, SHOW CREATE POLICY", TABLE, SHOW_ACCESS) \ M(SHOW_QUOTAS, "SHOW CREATE QUOTA", GLOBAL, SHOW_ACCESS) \ M(SHOW_SETTINGS_PROFILES, "SHOW PROFILES, SHOW CREATE SETTINGS PROFILE, SHOW CREATE PROFILE", GLOBAL, SHOW_ACCESS) \ M(SHOW_ACCESS, "", GROUP, ACCESS_MANAGEMENT) \ diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 2b4fbf411fe..3f3abff4e87 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -45,7 +45,15 @@ TEST(AccessRights, Union) lhs.grant(AccessType::INSERT); rhs.grant(AccessType::ALL, "db1"); lhs.makeUnion(rhs); - ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*"); + ASSERT_EQ(lhs.toString(), + "GRANT INSERT ON *.*, " + "GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, " + "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, " + "TRUNCATE, OPTIMIZE, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " + "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " + "SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " + "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " + "SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*"); } diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index c88e9c299a8..72b4b149bd7 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -45,22 +46,24 @@ namespace BlockIO InterpreterCreateRowPolicyQuery::execute() { auto & query = query_ptr->as(); - auto & access_control = getContext()->getAccessControl(); - getContext()->checkAccess(query.alter ? AccessType::ALTER_ROW_POLICY : AccessType::CREATE_ROW_POLICY); + auto required_access = getRequiredAccess(); if (!query.cluster.empty()) { query.replaceCurrentUserTag(getContext()->getUserName()); - return executeDDLQueryOnCluster(query_ptr, getContext()); + return executeDDLQueryOnCluster(query_ptr, getContext(), required_access); } assert(query.names->cluster.empty()); + auto & access_control = getContext()->getAccessControl(); + getContext()->checkAccess(required_access); + + query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); + std::optional roles_from_query; if (query.roles) roles_from_query = RolesOrUsersSet{*query.roles, access_control, getContext()->getUserID()}; - query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); - if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -105,4 +108,15 @@ void InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(RowPolicy & polic updateRowPolicyFromQueryImpl(policy, query, {}, {}); } + +AccessRightsElements InterpreterCreateRowPolicyQuery::getRequiredAccess() const +{ + const auto & query = query_ptr->as(); + AccessRightsElements res; + auto access_type = (query.alter ? AccessType::ALTER_ROW_POLICY : AccessType::CREATE_ROW_POLICY); + for (const auto & row_policy_name : query.names->full_names) + res.emplace_back(access_type, row_policy_name.database, row_policy_name.table_name); + return res; +} + } diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.h b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.h index 8adfe6b0855..e76cc1c165d 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.h +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.h @@ -6,8 +6,8 @@ namespace DB { - class ASTCreateRowPolicyQuery; +class AccessRightsElements; struct RowPolicy; class InterpreterCreateRowPolicyQuery : public IInterpreter, WithMutableContext @@ -20,6 +20,8 @@ public: static void updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query); private: + AccessRightsElements getRequiredAccess() const; + ASTPtr query_ptr; }; diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index 4d2e880561e..3437e7fe0f4 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -49,12 +49,37 @@ AccessRightsElements InterpreterDropAccessEntityQuery::getRequiredAccess() const AccessRightsElements res; switch (query.type) { - case AccessEntityType::USER: res.emplace_back(AccessType::DROP_USER); return res; - case AccessEntityType::ROLE: res.emplace_back(AccessType::DROP_ROLE); return res; - case AccessEntityType::SETTINGS_PROFILE: res.emplace_back(AccessType::DROP_SETTINGS_PROFILE); return res; - case AccessEntityType::ROW_POLICY: res.emplace_back(AccessType::DROP_ROW_POLICY); return res; - case AccessEntityType::QUOTA: res.emplace_back(AccessType::DROP_QUOTA); return res; - case AccessEntityType::MAX: break; + case AccessEntityType::USER: + { + res.emplace_back(AccessType::DROP_USER); + return res; + } + case AccessEntityType::ROLE: + { + res.emplace_back(AccessType::DROP_ROLE); + return res; + } + case AccessEntityType::SETTINGS_PROFILE: + { + res.emplace_back(AccessType::DROP_SETTINGS_PROFILE); + return res; + } + case AccessEntityType::ROW_POLICY: + { + if (query.row_policy_names) + { + for (const auto & row_policy_name : query.row_policy_names->full_names) + res.emplace_back(AccessType::DROP_ROW_POLICY, row_policy_name.database, row_policy_name.table_name); + } + return res; + } + case AccessEntityType::QUOTA: + { + res.emplace_back(AccessType::DROP_QUOTA); + return res; + } + case AccessEntityType::MAX: + break; } throw Exception( toString(query.type) + ": type is not supported by DROP query", ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.h b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.h index 0ee478e904e..ea2d127913f 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.h +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.h @@ -6,7 +6,6 @@ namespace DB { - class AccessRightsElements; class InterpreterDropAccessEntityQuery : public IInterpreter, WithMutableContext diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index 163cb57cab5..27345218e07 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -377,12 +377,48 @@ AccessRightsElements InterpreterShowCreateAccessEntityQuery::getRequiredAccess() AccessRightsElements res; switch (show_query.type) { - case AccessEntityType::USER: res.emplace_back(AccessType::SHOW_USERS); return res; - case AccessEntityType::ROLE: res.emplace_back(AccessType::SHOW_ROLES); return res; - case AccessEntityType::SETTINGS_PROFILE: res.emplace_back(AccessType::SHOW_SETTINGS_PROFILES); return res; - case AccessEntityType::ROW_POLICY: res.emplace_back(AccessType::SHOW_ROW_POLICIES); return res; - case AccessEntityType::QUOTA: res.emplace_back(AccessType::SHOW_QUOTAS); return res; - case AccessEntityType::MAX: break; + case AccessEntityType::USER: + { + res.emplace_back(AccessType::SHOW_USERS); + return res; + } + case AccessEntityType::ROLE: + { + res.emplace_back(AccessType::SHOW_ROLES); + return res; + } + case AccessEntityType::SETTINGS_PROFILE: + { + res.emplace_back(AccessType::SHOW_SETTINGS_PROFILES); + return res; + } + case AccessEntityType::ROW_POLICY: + { + if (show_query.row_policy_names) + { + for (const auto & row_policy_name : show_query.row_policy_names->full_names) + res.emplace_back(AccessType::SHOW_ROW_POLICIES, row_policy_name.database, row_policy_name.table_name); + } + else if (show_query.database_and_table_name) + { + if (show_query.database_and_table_name->second.empty()) + res.emplace_back(AccessType::SHOW_ROW_POLICIES, show_query.database_and_table_name->first); + else + res.emplace_back(AccessType::SHOW_ROW_POLICIES, show_query.database_and_table_name->first, show_query.database_and_table_name->second); + } + else + { + res.emplace_back(AccessType::SHOW_ROW_POLICIES); + } + return res; + } + case AccessEntityType::QUOTA: + { + res.emplace_back(AccessType::SHOW_QUOTAS); + return res; + } + case AccessEntityType::MAX: + break; } throw Exception(toString(show_query.type) + ": type is not supported by SHOW CREATE query", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index 89e07fecb0a..196141f9bfe 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -150,8 +150,11 @@ def test_grant_all_on_table(): instance.query("CREATE USER A, B") instance.query("GRANT ALL ON test.table TO A WITH GRANT OPTION") instance.query("GRANT ALL ON test.table TO B", user='A') - assert instance.query( - "SHOW GRANTS FOR B") == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" + assert instance.query("SHOW GRANTS FOR B") ==\ + "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, "\ + "DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, "\ + "SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, "\ + "SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" instance.query("REVOKE ALL ON test.table FROM B", user='A') assert instance.query("SHOW GRANTS FOR B") == "" diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 66a35bea06b..0a7f6958b4a 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -389,6 +389,52 @@ def test_dcl_management(): assert node.query("SHOW POLICIES") == "" +def test_grant_create_row_policy(): + copy_policy_xml('no_filters.xml') + assert node.query("SHOW POLICIES") == "" + node.query("CREATE USER X") + + expected_error = "necessary to have grant CREATE ROW POLICY ON mydb.filtered_table1" + assert expected_error in node.query_and_get_error("CREATE POLICY pA ON mydb.filtered_table1 FOR SELECT USING a Date: Fri, 11 Feb 2022 14:34:20 +0800 Subject: [PATCH 092/164] Update buildPushingToViewsChain.h typo --- src/Processors/Transforms/buildPushingToViewsChain.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 260fdfb3a19..98e7f19a37a 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -18,7 +18,7 @@ namespace DB struct ViewRuntimeData { - /// A query we should run over inserted block befire pushing into inner storage. + /// A query we should run over inserted block before pushing into inner storage. const ASTPtr query; /// This structure is expected by inner storage. Will convert query result to it. Block sample_block; From 3a4c2938d36465e6c8f6d45931d7aaaa616a9995 Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 11 Feb 2022 14:43:28 +0800 Subject: [PATCH 093/164] Translate zh/engines/database-engines/postgresql: sync translate from en doc --- docs/zh/engines/database-engines/postgresql.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/zh/engines/database-engines/postgresql.md b/docs/zh/engines/database-engines/postgresql.md index 12b8133f404..936216e8f5c 100644 --- a/docs/zh/engines/database-engines/postgresql.md +++ b/docs/zh/engines/database-engines/postgresql.md @@ -24,6 +24,7 @@ ENGINE = PostgreSQL('host:port', 'database', 'user', 'password'[, `use_table_cac - `database` — 远程数据库名次 - `user` — PostgreSQL用户名称 - `password` — PostgreSQL用户密码 + `schema` - PostgreSQL 模式 - `use_table_cache` — 定义数据库表结构是否已缓存或不进行。可选的。默认值: `0`. ## 支持的数据类型 {#data_types-support} From 6a57b30983421c1dbc5cc801f3490ce83c63445a Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 11 Feb 2022 14:44:42 +0800 Subject: [PATCH 094/164] Translate zh/engines/database-engines/postgresql: fix symbol --- docs/zh/engines/database-engines/postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/engines/database-engines/postgresql.md b/docs/zh/engines/database-engines/postgresql.md index 936216e8f5c..4d2af9182f9 100644 --- a/docs/zh/engines/database-engines/postgresql.md +++ b/docs/zh/engines/database-engines/postgresql.md @@ -24,7 +24,7 @@ ENGINE = PostgreSQL('host:port', 'database', 'user', 'password'[, `use_table_cac - `database` — 远程数据库名次 - `user` — PostgreSQL用户名称 - `password` — PostgreSQL用户密码 - `schema` - PostgreSQL 模式 +- `schema` - PostgreSQL 模式 - `use_table_cache` — 定义数据库表结构是否已缓存或不进行。可选的。默认值: `0`. ## 支持的数据类型 {#data_types-support} From a7c2cb53d8705b24866a1598323665efbb2ed055 Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 11 Feb 2022 16:58:13 +0800 Subject: [PATCH 095/164] Translate zh/engines/database-engines/replicated: sync translate --- docs/zh/engines/database-engines/replicated.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/zh/engines/database-engines/replicated.md b/docs/zh/engines/database-engines/replicated.md index 9ffebe04571..bd5841491dd 100644 --- a/docs/zh/engines/database-engines/replicated.md +++ b/docs/zh/engines/database-engines/replicated.md @@ -31,6 +31,7 @@ CREATE DATABASE testdb ENGINE = Replicated('zoo_path', 'shard_name', 'replica_na 当创建数据库的新副本时,该副本会自己创建表。如果副本已经不可用很长一段时间,并且已经滞后于复制日志-它用ZooKeeper中的当前元数据检查它的本地元数据,将带有数据的额外表移动到一个单独的非复制数据库(以免意外地删除任何多余的东西),创建缺失的表,如果表名已经被重命名,则更新表名。数据在`ReplicatedMergeTree`级别被复制,也就是说,如果表没有被复制,数据将不会被复制(数据库只负责元数据)。 +允许[`ALTER TABLE ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md)查询,但不允许复制。数据库引擎将只向当前副本添加/获取/删除分区/部件。但是,如果表本身使用了Replicated表引擎,那么数据将在使用`ATTACH`后被复制。 ## 使用示例 {#usage-example} 创建三台主机的集群: From e07c5751fac53bb255e54e98d5e1b7acad659b83 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Fri, 11 Feb 2022 17:18:55 +0800 Subject: [PATCH 096/164] No lock on append_entries --- src/Coordination/KeeperServer.cpp | 5 +---- src/Coordination/KeeperServer.h | 2 -- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 25d57e64e0a..558b28f9d46 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -286,10 +286,7 @@ RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForS for (const auto & [session_id, request] : requests_for_sessions) entries.push_back(getZooKeeperLogEntry(session_id, request)); - { - std::lock_guard lock(append_entries_mutex); - return raft_instance->append_entries(entries); - } + return raft_instance->append_entries(entries); } bool KeeperServer::isLeader() const diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 1fb02bb0987..4ed88ceb855 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -28,8 +28,6 @@ private: nuraft::ptr asio_service; nuraft::ptr asio_listener; - std::mutex append_entries_mutex; - std::mutex initialized_mutex; std::atomic initialized_flag = false; std::condition_variable initialized_cv; From c8cddd70772ea16c1cdf049b42864314e4e5697a Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 11 Feb 2022 17:45:49 +0800 Subject: [PATCH 097/164] Fix compression for URL engine --- src/IO/WriteBufferFromHTTP.cpp | 9 +++++++++ src/IO/WriteBufferFromHTTP.h | 2 ++ src/Storages/StorageURL.cpp | 2 +- 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 5ddc28d2db1..144678dfc5f 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -10,6 +10,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( const Poco::URI & uri, const std::string & method, const std::string & content_type, + const CompressionMethod compression_method, const ConnectionTimeouts & timeouts, size_t buffer_size_) : WriteBufferFromOStream(buffer_size_) @@ -24,6 +25,10 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( request.set("Content-Type", content_type); } + std::string encoding = toContentEncodingName(compression_method); + if (!encoding.empty()) + request.set("Content-Encoding", encoding); + LOG_TRACE((&Poco::Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString()); ostr = &session->sendRequest(request); @@ -31,6 +36,10 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( void WriteBufferFromHTTP::finalizeImpl() { + // for compressed body, the data is stored in buffered first + // here, make sure the content in the buffer has been flushed + this->nextImpl(); + receiveResponse(*session, request, response, false); /// TODO: Response body is ignored. } diff --git a/src/IO/WriteBufferFromHTTP.h b/src/IO/WriteBufferFromHTTP.h index 31b2a921889..9392282e016 100644 --- a/src/IO/WriteBufferFromHTTP.h +++ b/src/IO/WriteBufferFromHTTP.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -21,6 +22,7 @@ public: explicit WriteBufferFromHTTP(const Poco::URI & uri, const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST or PUT only const std::string & content_type = "", + const CompressionMethod compression_method = CompressionMethod::None, const ConnectionTimeouts & timeouts = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 32ab126faa9..7e282a9e568 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -303,7 +303,7 @@ StorageURLSink::StorageURLSink( std::string content_type = FormatFactory::instance().getContentType(format, context, format_settings); write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(Poco::URI(uri), http_method, content_type, timeouts), + std::make_unique(Poco::URI(uri), http_method, content_type, compression_method, timeouts), compression_method, 3); writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, {} /* write callback */, format_settings); From 032c7ea758262c65a1ca7e14de0b7ed63c0becbb Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 11 Feb 2022 17:46:33 +0800 Subject: [PATCH 098/164] Add description of compression in the doc --- docs/en/engines/table-engines/special/url.md | 33 ++++++++++++++------ 1 file changed, 24 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 04f035206b5..2d88aacf7e1 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -7,18 +7,33 @@ toc_title: URL Queries data to/from a remote HTTP/HTTPS server. This engine is similar to the [File](../../../engines/table-engines/special/file.md) engine. -Syntax: `URL(URL, Format)` +Syntax: `URL(URL [,Format] [,CompressionMethod])` + +- The `URL` parameter must conform to the structure of a Uniform Resource Locator. The specified URL must point to a server + that uses HTTP or HTTPS. This does not require any + additional headers for getting a response from the server. + +- The `Format` must be one that ClickHouse can use in + `SELECT` queries and, if necessary, in `INSERTs`. For the full list of supported formats, see + [Formats](../../../interfaces/formats.md#formats). + +- `CompressionMethod` indicates that whether the HTTP body should be compressed. If the compression is enabled, the HTTP packets sent by the URL engine contain 'Content-Encoding' header to indicate which compression method is used. + + To enable compression, please first make sure the remote HTTP endpoint indicated by the `URL` parameter supports corresponding compression algorithm. + + The supported `CompressionMethod` should be one of following + - gzip or gz + - deflate + - brotli or br + - lzma or xz + - zstd or zst + - lz4 + - bz2 + - snappy + - none ## Usage {#using-the-engine-in-the-clickhouse-server} -The `format` must be one that ClickHouse can use in -`SELECT` queries and, if necessary, in `INSERTs`. For the full list of supported formats, see -[Formats](../../../interfaces/formats.md#formats). - -The `URL` must conform to the structure of a Uniform Resource Locator. The specified URL must point to a server -that uses HTTP or HTTPS. This does not require any -additional headers for getting a response from the server. - `INSERT` and `SELECT` queries are transformed to `POST` and `GET` requests, respectively. For processing `POST` requests, the remote server must support [Chunked transfer encoding](https://en.wikipedia.org/wiki/Chunked_transfer_encoding). From 4a9d0c84df0a1e65dea96ec1ebd893d505c32a8d Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 11 Feb 2022 13:59:13 +0300 Subject: [PATCH 099/164] Update InterpreterCreateQuery.cpp --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 5be320f267d..4e0561ad750 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -662,7 +662,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } /// We can have queries like "CREATE TABLE ENGINE=" if /// supports schema inference (will determine table structure in it's constructor). - else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name)) + else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name)) // NOLINT throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); /// Even if query has list of columns, canonicalize it (unfold Nested columns). From f423c035d9f157f540dba2532f327a8db4abc128 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Feb 2022 14:10:15 +0300 Subject: [PATCH 100/164] Fix test --- .../0_stateless/02211_jsonl_format_extension.reference | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.reference b/tests/queries/0_stateless/02211_jsonl_format_extension.reference index 7b36cc96f5e..8b1acc12b63 100644 --- a/tests/queries/0_stateless/02211_jsonl_format_extension.reference +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.reference @@ -8,13 +8,3 @@ 7 8 9 -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 From bcd077fdc3f8b7c2b1ae4544cb9076b4146fbca3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 11 Feb 2022 21:20:57 +0800 Subject: [PATCH 101/164] add stateless test --- .../02206_format_override.reference | 33 +++++++++++++++++++ .../0_stateless/02206_format_override.sh | 25 ++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 tests/queries/0_stateless/02206_format_override.reference create mode 100644 tests/queries/0_stateless/02206_format_override.sh diff --git a/tests/queries/0_stateless/02206_format_override.reference b/tests/queries/0_stateless/02206_format_override.reference new file mode 100644 index 00000000000..e1bb01eeb2f --- /dev/null +++ b/tests/queries/0_stateless/02206_format_override.reference @@ -0,0 +1,33 @@ +File generated: +Options: --input-format=CSV --output-format JSONEachRow --format TSV +{"num1":"0","num2":"0"} +{"num1":"1","num2":"2"} +{"num1":"2","num2":"4"} +{"num1":"3","num2":"6"} +{"num1":"4","num2":"8"} +{"num1":"5","num2":"10"} +{"num1":"6","num2":"12"} +Options: --input-format=CSV --format TSV +0 0 +1 2 +2 4 +3 6 +4 8 +5 10 +6 12 +Options: --output-format=JSONEachRow --format CSV +{"num1":"0","num2":"0"} +{"num1":"1","num2":"2"} +{"num1":"2","num2":"4"} +{"num1":"3","num2":"6"} +{"num1":"4","num2":"8"} +{"num1":"5","num2":"10"} +{"num1":"6","num2":"12"} +Options: --format CSV +0,0 +1,2 +2,4 +3,6 +4,8 +5,10 +6,12 diff --git a/tests/queries/0_stateless/02206_format_override.sh b/tests/queries/0_stateless/02206_format_override.sh new file mode 100644 index 00000000000..1359f1edeb8 --- /dev/null +++ b/tests/queries/0_stateless/02206_format_override.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +SAMPLE_FILE="$CURDIR/02206_sample_data.csv" + +echo 'File generated:' +${CLICKHOUSE_LOCAL} -q "SELECT number, number * 2 from numbers(7) FORMAT TSV" | tr '\t' ',' >"$SAMPLE_FILE" + + +echo "Options: --input-format=CSV --output-format JSONEachRow --format TSV" +cat "$SAMPLE_FILE" | ${CLICKHOUSE_LOCAL} --input-format CSV --output-format JSONEachRow --format TSV --structure='num1 Int64, num2 Int64' --query='SELECT * from table' + +echo "Options: --input-format=CSV --format TSV" +cat "$SAMPLE_FILE" | ${CLICKHOUSE_LOCAL} --input-format CSV --format TSV --structure='num1 Int64, num2 Int64' --query='SELECT * from table' + +echo "Options: --output-format=JSONEachRow --format CSV" +cat "$SAMPLE_FILE" | ${CLICKHOUSE_LOCAL} --output-format JSONEachRow --format CSV --structure='num1 Int64, num2 Int64' --query='SELECT * from table' + +echo "Options: --format CSV" +cat "$SAMPLE_FILE" | ${CLICKHOUSE_LOCAL} --format CSV --structure='num1 Int64, num2 Int64' --query='SELECT * from table' + +rm "$SAMPLE_FILE" \ No newline at end of file From 2fcd69baf7cde250728e943926256c628144f179 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 11 Feb 2022 17:11:48 +0300 Subject: [PATCH 102/164] fix comparison with integers and floats in index analysis --- src/Storages/MergeTree/KeyCondition.cpp | 4 +-- .../02207_key_condition_floats.reference | 9 +++++ .../02207_key_condition_floats.sql | 34 +++++++++++++++++++ 3 files changed, 45 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02207_key_condition_floats.reference create mode 100644 tests/queries/0_stateless/02207_key_condition_floats.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 8ce65211e3e..c3321cd2682 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1292,8 +1292,8 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, key_expr_type_not_null = key_expr_type; bool cast_not_needed = is_set_const /// Set args are already casted inside Set::createFromAST - || ((isNativeNumber(key_expr_type_not_null) || isDateTime(key_expr_type_not_null)) - && (isNativeNumber(const_type) || isDateTime(const_type))); /// Numbers and DateTime are accurately compared without cast. + || ((isInteger(key_expr_type_not_null) || isDateTime(key_expr_type_not_null)) + && (isInteger(const_type) || isDateTime(const_type))); /// Integers and DateTime are accurately compared without cast. if (!cast_not_needed && !key_expr_type_not_null->equals(*const_type)) { diff --git a/tests/queries/0_stateless/02207_key_condition_floats.reference b/tests/queries/0_stateless/02207_key_condition_floats.reference new file mode 100644 index 00000000000..6c78023f8c0 --- /dev/null +++ b/tests/queries/0_stateless/02207_key_condition_floats.reference @@ -0,0 +1,9 @@ +2 +2 +2 +2 +2 +2 +2 +2 +1 diff --git a/tests/queries/0_stateless/02207_key_condition_floats.sql b/tests/queries/0_stateless/02207_key_condition_floats.sql new file mode 100644 index 00000000000..65527c65290 --- /dev/null +++ b/tests/queries/0_stateless/02207_key_condition_floats.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS t_key_condition_float; + +CREATE TABLE t_key_condition_float (a Float32) +ENGINE = MergeTree ORDER BY a; + +INSERT INTO t_key_condition_float VALUES (0.1), (0.2); + +SELECT count() FROM t_key_condition_float WHERE a > 0; +SELECT count() FROM t_key_condition_float WHERE a > 0.0; +SELECT count() FROM t_key_condition_float WHERE a > 0::Float32; +SELECT count() FROM t_key_condition_float WHERE a > 0::Float64; + +DROP TABLE t_key_condition_float; + +CREATE TABLE t_key_condition_float (a Float64) +ENGINE = MergeTree ORDER BY a; + +INSERT INTO t_key_condition_float VALUES (0.1), (0.2); + +SELECT count() FROM t_key_condition_float WHERE a > 0; +SELECT count() FROM t_key_condition_float WHERE a > 0.0; +SELECT count() FROM t_key_condition_float WHERE a > 0::Float32; +SELECT count() FROM t_key_condition_float WHERE a > 0::Float64; + +DROP TABLE t_key_condition_float; + +CREATE TABLE t_key_condition_float (a UInt64) +ENGINE = MergeTree ORDER BY a; + +INSERT INTO t_key_condition_float VALUES (1), (2); + +SELECT count() FROM t_key_condition_float WHERE a > 1.5; + +DROP TABLE t_key_condition_float; From 6a8e35930ffc25f0574179c5ad93635cfe073121 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 11 Feb 2022 18:20:37 +0300 Subject: [PATCH 103/164] fix comparison with integers and floats in index analysis --- src/Storages/MergeTree/KeyCondition.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index c3321cd2682..323b59e2902 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1292,8 +1292,8 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, key_expr_type_not_null = key_expr_type; bool cast_not_needed = is_set_const /// Set args are already casted inside Set::createFromAST - || ((isInteger(key_expr_type_not_null) || isDateTime(key_expr_type_not_null)) - && (isInteger(const_type) || isDateTime(const_type))); /// Integers and DateTime are accurately compared without cast. + || ((isNativeInteger(key_expr_type_not_null) || isDateTime(key_expr_type_not_null)) + && (isNativeInteger(const_type) || isDateTime(const_type))); /// Native integers and DateTime are accurately compared without cast. if (!cast_not_needed && !key_expr_type_not_null->equals(*const_type)) { From 1b1200f13a6b0ada7f84fba65222d901e9964ea1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 11 Feb 2022 16:36:45 +0100 Subject: [PATCH 104/164] Update in.md --- docs/en/sql-reference/operators/in.md | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 3632ab70200..0611b59c171 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -220,13 +220,12 @@ It also makes sense to specify a local table in the `GLOBAL IN` clause, in case You can use [`max_rows_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) and [`max_bytes_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) to control how much data is tranferred during distributed queries. -This is specially important if the `global in` query returns a large amount of data. Consider the following sql - -```sql + This is specially important if the `global in` query returns a large amount of data. Consider the following sql - + ```sql + select * from table1 where col1 global in (select col1 from table2 where ) + ``` -select * from table1 where col1 global in (select col1 from table2 where ) - -``` -If `some_predicate` is not selective enough, it will return large amount of data and cause performance issues. In such cases, it is wise to limit the data transfer over the network. Also, note that `set_overflow_mode` is set to `throw` (by default) meaning that an expception is raised when these thresolds are met + If `some_predicate` is not selective enough, it will return large amount of data and cause performance issues. In such cases, it is wise to limit the data transfer over the network. Also, note that [`set_overflow_mode`](../../operations/settings/query-complexity.md#set_overflow_mode) is set to `throw` (by default) meaning that an exception is raised when these thresholds are met. ### Distributed Subqueries and max_parallel_replicas {#max_parallel_replica-subqueries} From ea47b3082e57eaf70ef6851a7b0e88098a12f96e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 11 Feb 2022 16:38:01 +0100 Subject: [PATCH 105/164] Update in.md --- docs/en/sql-reference/operators/in.md | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 0611b59c171..d8468370f3e 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -220,13 +220,12 @@ It also makes sense to specify a local table in the `GLOBAL IN` clause, in case You can use [`max_rows_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) and [`max_bytes_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) to control how much data is tranferred during distributed queries. - This is specially important if the `global in` query returns a large amount of data. Consider the following sql - - ```sql - select * from table1 where col1 global in (select col1 from table2 where ) - ``` - - If `some_predicate` is not selective enough, it will return large amount of data and cause performance issues. In such cases, it is wise to limit the data transfer over the network. Also, note that [`set_overflow_mode`](../../operations/settings/query-complexity.md#set_overflow_mode) is set to `throw` (by default) meaning that an exception is raised when these thresholds are met. - +This is specially important if the `global in` query returns a large amount of data. Consider the following sql - +```sql +select * from table1 where col1 global in (select col1 from table2 where ) +``` + +If `some_predicate` is not selective enough, it will return large amount of data and cause performance issues. In such cases, it is wise to limit the data transfer over the network. Also, note that [`set_overflow_mode`](../../operations/settings/query-complexity.md#set_overflow_mode) is set to `throw` (by default) meaning that an exception is raised when these thresholds are met. ### Distributed Subqueries and max_parallel_replicas {#max_parallel_replica-subqueries} From bc905d969b61d77b75f4a544aa447fca09e3cb40 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 11 Feb 2022 16:51:45 +0100 Subject: [PATCH 106/164] add BeforeLambdaBody to .clang-format --- .clang-format | 1 + 1 file changed, 1 insertion(+) diff --git a/.clang-format b/.clang-format index c8b9672dc7d..d8f273702c8 100644 --- a/.clang-format +++ b/.clang-format @@ -12,6 +12,7 @@ BraceWrapping: AfterUnion: true BeforeCatch: true BeforeElse: true + BeforeLambdaBody: true IndentBraces: false BreakConstructorInitializersBeforeComma: false Cpp11BracedListStyle: true From c19d0a34948b249141b804cf453fb52cbb0a8241 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 8 Feb 2022 15:42:48 +0100 Subject: [PATCH 107/164] Update performance-comparison, fix x86 hard-code --- docker/test/performance-comparison/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/Dockerfile b/docker/test/performance-comparison/Dockerfile index eddaf969f33..fb47ed0cefa 100644 --- a/docker/test/performance-comparison/Dockerfile +++ b/docker/test/performance-comparison/Dockerfile @@ -1,5 +1,5 @@ # docker build -t clickhouse/performance-comparison . -FROM ubuntu:18.04 +FROM ubuntu:20.04 # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" From dcecbbaf816c4e2ffcedbe6ea66a62089eea07e7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Feb 2022 11:27:30 +0100 Subject: [PATCH 108/164] Add argument --all for docker imabes builder --- tests/ci/docker_images_check.py | 33 ++++++++++++++++++++++++--------- tests/ci/docker_test.py | 6 +++++- 2 files changed, 29 insertions(+), 10 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index a908f5fe11c..14a307c20f7 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -6,7 +6,7 @@ import os import shutil import subprocess import time -from typing import List, Optional, Set, Tuple, Union +from typing import Dict, List, Optional, Set, Tuple, Union from github import Github @@ -23,6 +23,8 @@ NAME = "Push to Dockerhub (actions)" TEMP_PATH = os.path.join(RUNNER_TEMP, "docker_images_check") +ImagesDict = Dict[str, Dict[str, Union[str, List[str]]]] + class DockerImage: def __init__( @@ -65,9 +67,7 @@ class DockerImage: return f"DockerImage(path={self.path},repo={self.repo},parent={self.parent})" -def get_changed_docker_images( - pr_info: PRInfo, repo_path: str, image_file_path: str -) -> Set[DockerImage]: +def get_images_dict(repo_path: str, image_file_path: str) -> ImagesDict: images_dict = {} path_to_images_file = os.path.join(repo_path, image_file_path) if os.path.exists(path_to_images_file): @@ -78,6 +78,13 @@ def get_changed_docker_images( "Image file %s doesnt exists in repo %s", image_file_path, repo_path ) + return images_dict + + +def get_changed_docker_images( + pr_info: PRInfo, images_dict: ImagesDict +) -> Set[DockerImage]: + if not images_dict: return set() @@ -290,10 +297,15 @@ def parse_args() -> argparse.Namespace: default="clickhouse", help="docker hub repository prefix", ) + parser.add_argument( + "--all", + action="store_true", + help="rebuild all images", + ) parser.add_argument( "--image-path", type=str, - action="append", + nargs="*", help="list of image paths to build instead of using pr_info + diff URL, " "e.g. 'docker/packager/binary'", ) @@ -336,15 +348,18 @@ def main(): shutil.rmtree(TEMP_PATH) os.makedirs(TEMP_PATH) - if args.image_path: + images_dict = get_images_dict(GITHUB_WORKSPACE, "docker/images.json") + + if args.all: + pr_info = PRInfo() + pr_info.changed_files = set(images_dict.keys()) + elif args.image_path: pr_info = PRInfo() pr_info.changed_files = set(i for i in args.image_path) else: pr_info = PRInfo(need_changed_files=True) - changed_images = get_changed_docker_images( - pr_info, GITHUB_WORKSPACE, "docker/images.json" - ) + changed_images = get_changed_docker_images(pr_info, images_dict) logging.info("Has changed images %s", ", ".join([im.path for im in changed_images])) image_versions, result_version = gen_versions(pr_info, args.suffix) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 4392641b215..a5f3bc16ab5 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -23,7 +23,11 @@ class TestDockerImageCheck(unittest.TestCase): "docker/docs/builder", } images = sorted( - list(di.get_changed_docker_images(pr_info, "/", self.docker_images_path)) + list( + di.get_changed_docker_images( + pr_info, di.get_images_dict("/", self.docker_images_path) + ) + ) ) self.maxDiff = None expected = sorted( From dc9a22df00c6ca1761357fd31b8effebd6a84c1c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Feb 2022 11:44:50 +0100 Subject: [PATCH 109/164] Add nightly docker images rebuild --- .github/workflows/nightly.yml | 73 +++++++++++++++++++++++++++++++++++ 1 file changed, 73 insertions(+) create mode 100644 .github/workflows/nightly.yml diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml new file mode 100644 index 00000000000..2602b9c28d5 --- /dev/null +++ b/.github/workflows/nightly.yml @@ -0,0 +1,73 @@ +name: NightlyBuilds + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + +"on": + schedule: + - cron: '0 0 * * *' + +jobs: + DockerHubPushAarch64: + runs-on: [self-hosted, style-checker-aarch64] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Images check + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 docker_images_check.py --suffix aarch64 --all + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images_aarch64 + path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json + DockerHubPushAmd64: + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Images check + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 docker_images_check.py --suffix amd64 --all + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images_amd64 + path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json + DockerHubPush: + needs: [DockerHubPushAmd64, DockerHubPushAarch64] + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download changed aarch64 images + uses: actions/download-artifact@v2 + with: + name: changed_images_aarch64 + path: ${{ runner.temp }} + - name: Download changed amd64 images + uses: actions/download-artifact@v2 + with: + name: changed_images_amd64 + path: ${{ runner.temp }} + - name: Images check + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/changed_images.json From bd3812a462d2b8c75576d4cab45efd0b85f80b06 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Feb 2022 16:22:32 +0100 Subject: [PATCH 110/164] Add a comment --- docker/test/integration/runner/Dockerfile | 1 + tests/ci/docker_manifests_merge.py | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 1aad2ae6770..bd062296411 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -58,6 +58,7 @@ RUN apt-get update \ RUN dockerd --version; docker --version +# Architecture of the image when BuildKit/buildx is used ARG TARGETARCH # FIXME: psycopg2-binary is not available for aarch64, we skip it for now RUN test x$TARGETARCH = xarm64 || ( python3 -m pip install \ diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index c6814b911ff..82d012bfe1a 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -57,7 +57,7 @@ def parse_args() -> argparse.Namespace: args = parser.parse_args() if len(args.suffixes) < 2: - raise parser.error("more than two --suffix should be given") + parser.error("more than two --suffix should be given") return args @@ -81,6 +81,7 @@ def strip_suffix(suffix: str, images: Images) -> Images: def check_sources(to_merge: Dict[str, Images]) -> Images: + """get a dict {arch1: Images, arch2: Images}""" result = {} # type: Images first_suffix = "" for suffix, images in to_merge.items(): From 7ca684aa9cdf8a8c277512d0dc7259c5b972c079 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Feb 2022 16:49:13 +0100 Subject: [PATCH 111/164] Revert checks for architectures in Dockerfiles --- docker/test/integration/runner/Dockerfile | 7 ++----- docker/test/pvs/Dockerfile | 12 ++++-------- 2 files changed, 6 insertions(+), 13 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index bd062296411..22dd2e14456 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -58,10 +58,7 @@ RUN apt-get update \ RUN dockerd --version; docker --version -# Architecture of the image when BuildKit/buildx is used -ARG TARGETARCH -# FIXME: psycopg2-binary is not available for aarch64, we skip it for now -RUN test x$TARGETARCH = xarm64 || ( python3 -m pip install \ +RUN python3 -m pip install \ PyMySQL \ aerospike==4.0.0 \ avro==1.10.2 \ @@ -91,7 +88,7 @@ RUN test x$TARGETARCH = xarm64 || ( python3 -m pip install \ urllib3 \ requests-kerberos \ pyhdfs \ - azure-storage-blob ) + azure-storage-blob COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ diff --git a/docker/test/pvs/Dockerfile b/docker/test/pvs/Dockerfile index f484feecfd0..01cc7c97548 100644 --- a/docker/test/pvs/Dockerfile +++ b/docker/test/pvs/Dockerfile @@ -4,11 +4,7 @@ ARG FROM_TAG=latest FROM clickhouse/binary-builder:$FROM_TAG -# PVS studio doesn't support aarch64/arm64, so there is a check for it everywhere -# We'll produce an empty image for arm64 -ARG TARGETARCH - -RUN test x$TARGETARCH = xarm64 || ( apt-get update --yes \ +RUN apt-get update --yes \ && apt-get install \ bash \ wget \ @@ -21,7 +17,7 @@ RUN test x$TARGETARCH = xarm64 || ( apt-get update --yes \ libprotoc-dev \ libgrpc++-dev \ libc-ares-dev \ - --yes --no-install-recommends ) + --yes --no-install-recommends #RUN wget -nv -O - http://files.viva64.com/etc/pubkey.txt | sudo apt-key add - #RUN sudo wget -nv -O /etc/apt/sources.list.d/viva64.list http://files.viva64.com/etc/viva64.list @@ -33,7 +29,7 @@ RUN test x$TARGETARCH = xarm64 || ( apt-get update --yes \ ENV PKG_VERSION="pvs-studio-latest" -RUN test x$TARGETARCH = xarm64 || ( set -x \ +RUN set -x \ && export PUBKEY_HASHSUM="ad369a2e9d8b8c30f5a9f2eb131121739b79c78e03fef0f016ea51871a5f78cd4e6257b270dca0ac3be3d1f19d885516" \ && wget -nv https://files.viva64.com/etc/pubkey.txt -O /tmp/pubkey.txt \ && echo "${PUBKEY_HASHSUM} /tmp/pubkey.txt" | sha384sum -c \ @@ -41,7 +37,7 @@ RUN test x$TARGETARCH = xarm64 || ( set -x \ && wget -nv "https://files.viva64.com/${PKG_VERSION}.deb" \ && { debsig-verify ${PKG_VERSION}.deb \ || echo "WARNING: Some file was just downloaded from the internet without any validation and we are installing it into the system"; } \ - && dpkg -i "${PKG_VERSION}.deb" ) + && dpkg -i "${PKG_VERSION}.deb" ENV CCACHE_DIR=/test_output/ccache From 0f21981694acdc2d4a03fb86a14b6712baa841ec Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Feb 2022 17:34:55 +0100 Subject: [PATCH 112/164] Add a flag for images architecture --- docker/images.json | 4 ++ tests/ci/docker_images_check.py | 52 ++++++++++++++++++++--- tests/ci/docker_test.py | 69 ++++++++++++++++++++++++++----- tests/ci/tests/docker_images.json | 1 + 4 files changed, 109 insertions(+), 17 deletions(-) diff --git a/docker/images.json b/docker/images.json index 354bdaa8728..01284d4de69 100644 --- a/docker/images.json +++ b/docker/images.json @@ -32,6 +32,7 @@ "dependent": [] }, "docker/test/pvs": { + "only_amd64": true, "name": "clickhouse/pvs-test", "dependent": [] }, @@ -72,6 +73,7 @@ "dependent": [] }, "docker/test/integration/runner": { + "only_amd64": true, "name": "clickhouse/integration-tests-runner", "dependent": [] }, @@ -124,6 +126,7 @@ "dependent": [] }, "docker/test/integration/kerberos_kdc": { + "only_amd64": true, "name": "clickhouse/kerberos-kdc", "dependent": [] }, @@ -137,6 +140,7 @@ ] }, "docker/test/integration/kerberized_hadoop": { + "only_amd64": true, "name": "clickhouse/kerberized-hadoop", "dependent": [] }, diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 14a307c20f7..140ede3067f 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -3,6 +3,7 @@ import argparse import json import logging import os +import platform import shutil import subprocess import time @@ -23,7 +24,7 @@ NAME = "Push to Dockerhub (actions)" TEMP_PATH = os.path.join(RUNNER_TEMP, "docker_images_check") -ImagesDict = Dict[str, Dict[str, Union[str, List[str]]]] +ImagesDict = Dict[str, dict] class DockerImage: @@ -31,18 +32,24 @@ class DockerImage: self, path: str, repo: str, + only_amd64: bool, parent: Optional["DockerImage"] = None, gh_repo_path: str = GITHUB_WORKSPACE, ): self.path = path self.full_path = os.path.join(gh_repo_path, path) self.repo = repo + self.only_amd64 = only_amd64 self.parent = parent self.built = False def __eq__(self, other) -> bool: # type: ignore """Is used to check if DockerImage is in a set or not""" - return self.path == other.path and self.repo == self.repo + return ( + self.path == other.path + and self.repo == self.repo + and self.only_amd64 == other.only_amd64 + ) def __lt__(self, other) -> bool: if not isinstance(other, DockerImage): @@ -68,6 +75,7 @@ class DockerImage: def get_images_dict(repo_path: str, image_file_path: str) -> ImagesDict: + """Return images suppose to build on the current architecture host""" images_dict = {} path_to_images_file = os.path.join(repo_path, image_file_path) if os.path.exists(path_to_images_file): @@ -103,6 +111,7 @@ def get_changed_docker_images( for f in files_changed: if f.startswith(dockerfile_dir): name = image_description["name"] + only_amd64 = image_description.get("only_amd64", False) logging.info( "Found changed file '%s' which affects " "docker image '%s' with path '%s'", @@ -110,7 +119,7 @@ def get_changed_docker_images( name, dockerfile_dir, ) - changed_images.append(DockerImage(dockerfile_dir, name)) + changed_images.append(DockerImage(dockerfile_dir, name, only_amd64)) break # The order is important: dependents should go later than bases, so that @@ -125,9 +134,9 @@ def get_changed_docker_images( dependent, image, ) - changed_images.append( - DockerImage(dependent, images_dict[dependent]["name"], image) - ) + name = images_dict[dependent]["name"] + only_amd64 = images_dict[dependent].get("only_amd64", False) + changed_images.append(DockerImage(dependent, name, only_amd64, image)) index += 1 if index > 5 * len(images_dict): # Sanity check to prevent infinite loop. @@ -168,12 +177,43 @@ def gen_versions( return versions, result_version +def build_and_push_dummy_image( + image: DockerImage, + version_string: str, + push: bool, +) -> Tuple[bool, str]: + dummy_source = "ubuntu:20.04" + logging.info("Building docker image %s as %s", image.repo, dummy_source) + build_log = os.path.join( + TEMP_PATH, f"build_and_push_log_{image.repo.replace('/', '_')}_{version_string}" + ) + with open(build_log, "wb") as bl: + cmd = ( + f"docker pull {dummy_source}; " + f"docker tag {dummy_source} {image.repo}:{version_string}; " + ) + if push: + cmd += f"docker push {image.repo}:{version_string}" + + logging.info("Docker command to run: %s", cmd) + with subprocess.Popen(cmd, shell=True, stderr=bl, stdout=bl) as proc: + retcode = proc.wait() + + if retcode != 0: + return False, build_log + + logging.info("Processing of %s successfully finished", image.repo) + return True, build_log + + def build_and_push_one_image( image: DockerImage, version_string: str, push: bool, child: bool, ) -> Tuple[bool, str]: + if image.only_amd64 and platform.machine() not in ["amd64", "x86_64"]: + return build_and_push_dummy_image(image, version_string, push) logging.info( "Building docker image %s with version %s from path %s", image.repo, diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index a5f3bc16ab5..27bfe07db53 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -32,49 +32,60 @@ class TestDockerImageCheck(unittest.TestCase): self.maxDiff = None expected = sorted( [ - di.DockerImage("docker/test/base", "clickhouse/test-base"), - di.DockerImage("docker/docs/builder", "clickhouse/docs-builder"), + di.DockerImage("docker/test/base", "clickhouse/test-base", False), + di.DockerImage("docker/docs/builder", "clickhouse/docs-builder", True), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", + False, "clickhouse/test-base", ), di.DockerImage( "docker/test/integration/base", "clickhouse/integration-test", + False, "clickhouse/test-base", ), di.DockerImage( - "docker/test/fuzzer", "clickhouse/fuzzer", "clickhouse/test-base" + "docker/test/fuzzer", + "clickhouse/fuzzer", + False, + "clickhouse/test-base", ), di.DockerImage( "docker/test/keeper-jepsen", "clickhouse/keeper-jepsen-test", + False, "clickhouse/test-base", ), di.DockerImage( "docker/docs/check", "clickhouse/docs-check", + False, "clickhouse/docs-builder", ), di.DockerImage( "docker/docs/release", "clickhouse/docs-release", + False, "clickhouse/docs-builder", ), di.DockerImage( "docker/test/stateful", "clickhouse/stateful-test", + False, "clickhouse/stateless-test", ), di.DockerImage( "docker/test/unit", "clickhouse/unit-test", + False, "clickhouse/stateless-test", ), di.DockerImage( "docker/test/stress", "clickhouse/stress-test", + False, "clickhouse/stateful-test", ), ] @@ -96,13 +107,15 @@ class TestDockerImageCheck(unittest.TestCase): @patch("builtins.open") @patch("subprocess.Popen") - def test_build_and_push_one_image(self, mock_popen, mock_open): + @patch("platform.machine") + def test_build_and_push_one_image(self, mock_machine, mock_popen, mock_open): mock_popen.return_value.__enter__.return_value.wait.return_value = 0 - image = di.DockerImage("path", "name", gh_repo_path="") + image = di.DockerImage("path", "name", False, gh_repo_path="") result, _ = di.build_and_push_one_image(image, "version", True, True) mock_open.assert_called_once() mock_popen.assert_called_once() + mock_machine.assert_not_called() self.assertIn( "docker buildx build --builder default --build-arg FROM_TAG=version " "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version --cache-from " @@ -110,11 +123,15 @@ class TestDockerImageCheck(unittest.TestCase): mock_popen.call_args.args, ) self.assertTrue(result) + mock_open.reset_mock() + mock_popen.reset_mock() + mock_machine.reset_mock() - mock_open.reset() - mock_popen.reset() mock_popen.return_value.__enter__.return_value.wait.return_value = 0 result, _ = di.build_and_push_one_image(image, "version2", False, True) + mock_open.assert_called_once() + mock_popen.assert_called_once() + mock_machine.assert_not_called() self.assertIn( "docker buildx build --builder default --build-arg FROM_TAG=version2 " "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version2 --cache-from " @@ -123,8 +140,14 @@ class TestDockerImageCheck(unittest.TestCase): ) self.assertTrue(result) + mock_open.reset_mock() + mock_popen.reset_mock() + mock_machine.reset_mock() mock_popen.return_value.__enter__.return_value.wait.return_value = 1 result, _ = di.build_and_push_one_image(image, "version2", False, False) + mock_open.assert_called_once() + mock_popen.assert_called_once() + mock_machine.assert_not_called() self.assertIn( "docker buildx build --builder default " "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version2 --cache-from " @@ -133,13 +156,37 @@ class TestDockerImageCheck(unittest.TestCase): ) self.assertFalse(result) + mock_open.reset_mock() + mock_popen.reset_mock() + mock_machine.reset_mock() + only_amd64_image = di.DockerImage("path", "name", True) + mock_popen.return_value.__enter__.return_value.wait.return_value = 0 + + result, _ = di.build_and_push_one_image(only_amd64_image, "version", True, True) + mock_open.assert_called_once() + mock_popen.assert_called_once() + mock_machine.assert_called_once() + self.assertIn( + "docker pull ubuntu:20.04; docker tag ubuntu:20.04 name:version; " + "docker push name:version", + mock_popen.call_args.args, + ) + self.assertTrue(result) + result, _ = di.build_and_push_one_image( + only_amd64_image, "version", False, True + ) + self.assertIn( + "docker pull ubuntu:20.04; docker tag ubuntu:20.04 name:version; ", + mock_popen.call_args.args, + ) + @patch("docker_images_check.build_and_push_one_image") def test_process_image_with_parents(self, mock_build): mock_build.side_effect = lambda w, x, y, z: (True, f"{w.repo}_{x}.log") - im1 = di.DockerImage("path1", "repo1") - im2 = di.DockerImage("path2", "repo2", im1) - im3 = di.DockerImage("path3", "repo3", im2) - im4 = di.DockerImage("path4", "repo4", im1) + im1 = di.DockerImage("path1", "repo1", False) + im2 = di.DockerImage("path2", "repo2", False, im1) + im3 = di.DockerImage("path3", "repo3", False, im2) + im4 = di.DockerImage("path4", "repo4", False, im1) # We use list to have determined order of image builgings images = [im4, im1, im3, im2, im1] results = [ diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images.json index 354bdaa8728..ca5c516bccb 100644 --- a/tests/ci/tests/docker_images.json +++ b/tests/ci/tests/docker_images.json @@ -150,6 +150,7 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", + "only_amd64": true, "dependent": [ "docker/docs/check", "docker/docs/release" From 65e8605af161ef065fd8afb503b61e247c45af4f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 11 Feb 2022 16:45:54 +0100 Subject: [PATCH 113/164] Make testflows aarch64 compatible --- docker/test/testflows/runner/Dockerfile | 33 ++++++++++++++----------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/docker/test/testflows/runner/Dockerfile b/docker/test/testflows/runner/Dockerfile index d15f237587b..fbff6fd5e97 100644 --- a/docker/test/testflows/runner/Dockerfile +++ b/docker/test/testflows/runner/Dockerfile @@ -43,24 +43,27 @@ RUN pip3 install urllib3 testflows==1.7.20 docker-compose==1.29.1 docker==5.0.0 ENV DOCKER_CHANNEL stable ENV DOCKER_VERSION 20.10.6 -RUN set -eux; \ - \ -# this "case" statement is generated via "update.sh" - \ - if ! wget -nv -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \ - echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${x86_64}'"; \ - exit 1; \ - fi; \ - \ - tar --extract \ +# Architecture of the image when BuildKit/buildx is used +ARG TARGETARCH + +# Install docker +RUN arch=${TARGETARCH:-amd64} \ + && case $arch in \ + amd64) rarch=x86_64 ;; \ + arm64) rarch=aarch64 ;; \ + esac \ + && set -eux \ + && if ! wget -nv -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/${rarch}/docker-${DOCKER_VERSION}.tgz"; then \ + echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${rarch}'" \ + && exit 1; \ + fi \ + && tar --extract \ --file docker.tgz \ --strip-components 1 \ --directory /usr/local/bin/ \ - ; \ - rm docker.tgz; \ - \ - dockerd --version; \ - docker --version + && rm docker.tgz \ + && dockerd --version \ + && docker --version COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ From ac459dccd86899ffb40f3a3362abef6848dc84bb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 11 Feb 2022 16:47:49 +0100 Subject: [PATCH 114/164] Force update kerberized hadoop --- docker/test/integration/kerberized_hadoop/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/kerberized_hadoop/Dockerfile b/docker/test/integration/kerberized_hadoop/Dockerfile index 025f4b27fde..e42d115999a 100644 --- a/docker/test/integration/kerberized_hadoop/Dockerfile +++ b/docker/test/integration/kerberized_hadoop/Dockerfile @@ -20,4 +20,4 @@ RUN cd /tmp && \ cd commons-daemon-1.0.15-src/src/native/unix && \ ./configure && \ make && \ - cp ./jsvc /usr/local/hadoop/sbin + cp ./jsvc /usr/local/hadoop-2.7.0/sbin From 50131fd73201bed1896dbed976bcd191e4fed48e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 11 Feb 2022 18:14:55 +0000 Subject: [PATCH 115/164] Fix cancelation for S3 and HDFS --- src/Storages/HDFS/StorageHDFS.cpp | 61 ++++++++++++++++--------------- src/Storages/StorageS3.cpp | 58 +++++++++++++++-------------- 2 files changed, 62 insertions(+), 57 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index d40838ad141..7b07e929c76 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -372,44 +372,47 @@ String HDFSSource::getName() const Chunk HDFSSource::generate() { - if (!reader) - return {}; - - Chunk chunk; - if (reader->pull(chunk)) + while (true) { - Columns columns = chunk.getColumns(); - UInt64 num_rows = chunk.getNumRows(); + if (!reader || isCancelled()) + break; - /// Enrich with virtual columns. - if (need_path_column) + Chunk chunk; + if (reader->pull(chunk)) { - auto column = DataTypeLowCardinality{std::make_shared()}.createColumnConst(num_rows, current_path); - columns.push_back(column->convertToFullColumnIfConst()); + Columns columns = chunk.getColumns(); + UInt64 num_rows = chunk.getNumRows(); + + /// Enrich with virtual columns. + if (need_path_column) + { + auto column = DataTypeLowCardinality{std::make_shared()}.createColumnConst(num_rows, current_path); + columns.push_back(column->convertToFullColumnIfConst()); + } + + if (need_file_column) + { + size_t last_slash_pos = current_path.find_last_of('/'); + auto file_name = current_path.substr(last_slash_pos + 1); + + auto column = DataTypeLowCardinality{std::make_shared()}.createColumnConst(num_rows, std::move(file_name)); + columns.push_back(column->convertToFullColumnIfConst()); + } + + return Chunk(std::move(columns), num_rows); } - if (need_file_column) { - size_t last_slash_pos = current_path.find_last_of('/'); - auto file_name = current_path.substr(last_slash_pos + 1); + std::lock_guard lock(reader_mutex); + reader.reset(); + pipeline.reset(); + read_buf.reset(); - auto column = DataTypeLowCardinality{std::make_shared()}.createColumnConst(num_rows, std::move(file_name)); - columns.push_back(column->convertToFullColumnIfConst()); + if (!initialize()) + break; } - - return Chunk(std::move(columns), num_rows); } - - { - std::lock_guard lock(reader_mutex); - reader.reset(); - pipeline.reset(); - read_buf.reset(); - - if (!initialize()) - return {}; - } - return generate(); + return {}; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2b9ba7585e1..9a85644d825 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -302,40 +302,42 @@ String StorageS3Source::getName() const Chunk StorageS3Source::generate() { - if (!reader) - return {}; - - Chunk chunk; - if (reader->pull(chunk)) + while (true) { - UInt64 num_rows = chunk.getNumRows(); + if (!reader || isCancelled()) + break; - if (with_path_column) - chunk.addColumn(DataTypeLowCardinality{std::make_shared()} - .createColumnConst(num_rows, file_path) - ->convertToFullColumnIfConst()); - if (with_file_column) + Chunk chunk; + if (reader->pull(chunk)) { - size_t last_slash_pos = file_path.find_last_of('/'); - chunk.addColumn(DataTypeLowCardinality{std::make_shared()} - .createColumnConst(num_rows, file_path.substr(last_slash_pos + 1)) - ->convertToFullColumnIfConst()); + UInt64 num_rows = chunk.getNumRows(); + + if (with_path_column) + chunk.addColumn(DataTypeLowCardinality{std::make_shared()} + .createColumnConst(num_rows, file_path) + ->convertToFullColumnIfConst()); + if (with_file_column) + { + size_t last_slash_pos = file_path.find_last_of('/'); + chunk.addColumn(DataTypeLowCardinality{std::make_shared()} + .createColumnConst(num_rows, file_path.substr(last_slash_pos + 1)) + ->convertToFullColumnIfConst()); + } + + return chunk; } - return chunk; + { + std::lock_guard lock(reader_mutex); + reader.reset(); + pipeline.reset(); + read_buf.reset(); + + if (!initialize()) + break; + } } - - { - std::lock_guard lock(reader_mutex); - reader.reset(); - pipeline.reset(); - read_buf.reset(); - - if (!initialize()) - return {}; - } - - return generate(); + return {}; } static bool checkIfObjectExists(const std::shared_ptr & client, const String & bucket, const String & key) From 14f3a11c01a49bd2cf052c3b33f8012eec589ed4 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 11 Feb 2022 14:38:11 -0400 Subject: [PATCH 116/164] Update mergetree.md --- .../table-engines/mergetree-family/mergetree.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 4448372c522..3f140f85396 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -872,3 +872,13 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' ``` Если диск сконфигурирован как `cold`, данные будут переноситься в S3 при срабатывании правил TTL или когда свободное место на локальном диске станет меньше порогового значения, которое определяется как `move_factor * disk_size`. + +## Виртуальные столбцы {#virtual-columns} + +- `_part` — Имя куска. +- `_part_index` — Номер куска по порядку в результате запроса. +- `_partition_id` — Имя партиции. +- `_part_uuid` — Уникальный идентификатор куска (если включена MergeTree настройка `assign_part_uuids`). +- `_partition_value` — Значения (кортеж) выражения `partition by`. +- `_sample_factor` — Коэффициент сэмплирования (из запроса). + From f935c524ea93e2c94b3a0f01360cceeea3a013ff Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 11 Feb 2022 15:28:20 -0400 Subject: [PATCH 117/164] fix description of MATERIALIZE COLUMN --- docs/ru/sql-reference/statements/alter/column.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index 4de2d067cce..7a97e4911b0 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -197,12 +197,13 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; ## MATERIALIZE COLUMN {#materialize-column} -Материализует столбец таблицы в кусках, в которых отсутствуют значения. Используется, если необходимо создать новый столбец со сложным материализованным выражением или выражением для заполнения по умолчанию (`DEFAULT`), потому как вычисление такого столбца прямо во время выполнения запроса `SELECT` оказывается ощутимо затратным. Чтобы совершить ту же операцию для существующего столбца, используйте модификатор `FINAL`. +Материализует или обновляет столбец таблицы с выражением для значения по умолчанию (`DEFAULT` или `MATERIALIZED`). +Используется, если необходимо добавить или обновить столбец со сложным выражением, потому как вычисление такого выражения прямо во время выполнения запроса `SELECT` оказывается ощутимо затратным. Синтаксис: ```sql -ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; +ALTER TABLE table MATERIALIZE COLUMN col; ``` **Пример** From 2daacc1a3ecd686347f341a74953e338d5b811aa Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 11 Feb 2022 15:40:20 -0400 Subject: [PATCH 118/164] Update column.md --- .../sql-reference/statements/alter/column.md | 33 ++++++++++++++----- 1 file changed, 24 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 2e562e20467..6bb63ea06a6 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -197,12 +197,13 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; ## MATERIALIZE COLUMN {#materialize-column} -Materializes the column in the parts where the column is missing. This is useful in case of creating a new column with complicated `DEFAULT` or `MATERIALIZED` expression. Calculation of the column directly on `SELECT` query can cause bigger request execution time, so it is reasonable to use `MATERIALIZE COLUMN` for such columns. To perform same manipulation for existing column, use `FINAL` modifier. +Materializes or updates a column with an expression for a default value (`DEFAULT` or `MATERIALIZED`). +It is used if it is necessary to add or update a column with a complicated expression, because evaluating such an expression directly on `SELECT` executing turns out to be expensive. Syntax: ```sql -ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; +ALTER TABLE table MATERIALIZE COLUMN col; ``` **Example** @@ -211,20 +212,34 @@ ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; DROP TABLE IF EXISTS tmp; SET mutations_sync = 2; CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple(); -INSERT INTO tmp SELECT * FROM system.numbers LIMIT 10; +INSERT INTO tmp SELECT * FROM system.numbers LIMIT 5; ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x); ALTER TABLE tmp MATERIALIZE COLUMN s; +SELECT groupArray(x), groupArray(s) FROM (select x,s from tmp order by x); + +┌─groupArray(x)─┬─groupArray(s)─────────┐ +│ [0,1,2,3,4] │ ['0','1','2','3','4'] │ +└───────────────┴───────────────────────┘ + +ALTER TABLE tmp MODIFY COLUMN s String MATERIALIZED toString(round(100/x)); + +INSERT INTO tmp SELECT * FROM system.numbers LIMIT 5,5; + SELECT groupArray(x), groupArray(s) FROM tmp; -``` -**Result:** +┌─groupArray(x)─────────┬─groupArray(s)──────────────────────────────────┐ +│ [0,1,2,3,4,5,6,7,8,9] │ ['0','1','2','3','4','20','17','14','12','11'] │ +└───────────────────────┴────────────────────────────────────────────────┘ -```sql -┌─groupArray(x)─────────┬─groupArray(s)─────────────────────────────┐ -│ [0,1,2,3,4,5,6,7,8,9] │ ['0','1','2','3','4','5','6','7','8','9'] │ -└───────────────────────┴───────────────────────────────────────────┘ +ALTER TABLE tmp MATERIALIZE COLUMN s; + +SELECT groupArray(x), groupArray(s) FROM tmp; + +┌─groupArray(x)─────────┬─groupArray(s)─────────────────────────────────────────┐ +│ [0,1,2,3,4,5,6,7,8,9] │ ['inf','100','50','33','25','20','17','14','12','11'] │ +└───────────────────────┴───────────────────────────────────────────────────────┘ ``` **See Also** From 7ad3e182ecac778cd544bf340f23efb4b18752ff Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 11 Feb 2022 15:40:57 -0400 Subject: [PATCH 119/164] Update column.md --- .../sql-reference/statements/alter/column.md | 28 ++++++++++++++----- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index 7a97e4911b0..1ab45fb6df8 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -212,20 +212,34 @@ ALTER TABLE table MATERIALIZE COLUMN col; DROP TABLE IF EXISTS tmp; SET mutations_sync = 2; CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple(); -INSERT INTO tmp SELECT * FROM system.numbers LIMIT 10; +INSERT INTO tmp SELECT * FROM system.numbers LIMIT 5; ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x); ALTER TABLE tmp MATERIALIZE COLUMN s; +SELECT groupArray(x), groupArray(s) FROM (select x,s from tmp order by x); + +┌─groupArray(x)─┬─groupArray(s)─────────┐ +│ [0,1,2,3,4] │ ['0','1','2','3','4'] │ +└───────────────┴───────────────────────┘ + +ALTER TABLE tmp MODIFY COLUMN s String MATERIALIZED toString(round(100/x)); + +INSERT INTO tmp SELECT * FROM system.numbers LIMIT 5,5; + SELECT groupArray(x), groupArray(s) FROM tmp; -``` -**Результат:** +┌─groupArray(x)─────────┬─groupArray(s)──────────────────────────────────┐ +│ [0,1,2,3,4,5,6,7,8,9] │ ['0','1','2','3','4','20','17','14','12','11'] │ +└───────────────────────┴────────────────────────────────────────────────┘ -```sql -┌─groupArray(x)─────────┬─groupArray(s)─────────────────────────────┐ -│ [0,1,2,3,4,5,6,7,8,9] │ ['0','1','2','3','4','5','6','7','8','9'] │ -└───────────────────────┴───────────────────────────────────────────┘ +ALTER TABLE tmp MATERIALIZE COLUMN s; + +SELECT groupArray(x), groupArray(s) FROM tmp; + +┌─groupArray(x)─────────┬─groupArray(s)─────────────────────────────────────────┐ +│ [0,1,2,3,4,5,6,7,8,9] │ ['inf','100','50','33','25','20','17','14','12','11'] │ +└───────────────────────┴───────────────────────────────────────────────────────┘ ``` ## Ограничения запроса ALTER {#ogranicheniia-zaprosa-alter} From 4554f0d542537cd374e7af4b9c8f2a8d3cd4503c Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 11 Feb 2022 15:45:28 -0400 Subject: [PATCH 120/164] Update column.md --- docs/ru/sql-reference/statements/alter/column.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index 1ab45fb6df8..fea4c00ac05 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -242,6 +242,10 @@ SELECT groupArray(x), groupArray(s) FROM tmp; └───────────────────────┴───────────────────────────────────────────────────────┘ ``` +**Смотрите также** + +- [MATERIALIZED](../../statements/create/table.md#materialized). + ## Ограничения запроса ALTER {#ogranicheniia-zaprosa-alter} Запрос `ALTER` позволяет создавать и удалять отдельные элементы (столбцы) вложенных структур данных, но не вложенные структуры данных целиком. Для добавления вложенной структуры данных, вы можете добавить столбцы с именем вида `name.nested_name` и типом `Array(T)` - вложенная структура данных полностью эквивалентна нескольким столбцам-массивам с именем, имеющим одинаковый префикс до точки. From 7beff9048ce74bdc34f8c96d4cc00e51d4f758e6 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 11 Feb 2022 16:50:53 -0400 Subject: [PATCH 121/164] Update settings.md --- docs/ru/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index affa90d9840..ba5fc63331a 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2119,7 +2119,7 @@ ClickHouse генерирует исключение: - 1 — включен режим параллельного разбора. - 0 — отключен режим параллельного разбора. -Значение по умолчанию: `0`. +Значение по умолчанию: `1`. ## output_format_parallel_formatting {#output-format-parallel-formatting} @@ -2130,7 +2130,7 @@ ClickHouse генерирует исключение: - 1 — включен режим параллельного форматирования. - 0 — отключен режим параллельного форматирования. -Значение по умолчанию: `0`. +Значение по умолчанию: `1`. ## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing} From a66b30f21a2e9439c452c78539f7654562a37b5b Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 11 Feb 2022 16:52:12 -0400 Subject: [PATCH 122/164] Update settings.md --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8a0fd618d32..986d5eadd80 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2304,7 +2304,7 @@ Possible values: - 1 — Enabled. - 0 — Disabled. -Default value: `0`. +Default value: `1`. ## output_format_parallel_formatting {#output-format-parallel-formatting} @@ -2315,7 +2315,7 @@ Possible values: - 1 — Enabled. - 0 — Disabled. -Default value: `0`. +Default value: `1`. ## min_chunk_bytes_for_parallel_parsing {#min-chunk-bytes-for-parallel-parsing} From 7b7bc8f6c37ee7ff384ab05583d275d43ecdb9fc Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Sat, 12 Feb 2022 10:43:53 +0800 Subject: [PATCH 123/164] Address review comment --- src/IO/WriteBufferFromHTTP.cpp | 7 +++---- src/IO/WriteBufferFromHTTP.h | 3 +-- src/Storages/StorageURL.cpp | 3 ++- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 144678dfc5f..622fab91fcc 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -10,7 +10,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( const Poco::URI & uri, const std::string & method, const std::string & content_type, - const CompressionMethod compression_method, + const std::string & content_encoding, const ConnectionTimeouts & timeouts, size_t buffer_size_) : WriteBufferFromOStream(buffer_size_) @@ -25,9 +25,8 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( request.set("Content-Type", content_type); } - std::string encoding = toContentEncodingName(compression_method); - if (!encoding.empty()) - request.set("Content-Encoding", encoding); + if (!content_encoding.empty()) + request.set("Content-Encoding", content_encoding); LOG_TRACE((&Poco::Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString()); diff --git a/src/IO/WriteBufferFromHTTP.h b/src/IO/WriteBufferFromHTTP.h index 9392282e016..6966bc8a5c5 100644 --- a/src/IO/WriteBufferFromHTTP.h +++ b/src/IO/WriteBufferFromHTTP.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -22,7 +21,7 @@ public: explicit WriteBufferFromHTTP(const Poco::URI & uri, const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST or PUT only const std::string & content_type = "", - const CompressionMethod compression_method = CompressionMethod::None, + const std::string & content_encoding = "", const ConnectionTimeouts & timeouts = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index baed961accf..508c9d8b157 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -346,9 +346,10 @@ StorageURLSink::StorageURLSink( : SinkToStorage(sample_block) { std::string content_type = FormatFactory::instance().getContentType(format, context, format_settings); + std::string content_encoding = toContentEncodingName(compression_method); write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(Poco::URI(uri), http_method, content_type, compression_method, timeouts), + std::make_unique(Poco::URI(uri), http_method, content_type, content_encoding, timeouts), compression_method, 3); writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, {} /* write callback */, format_settings); From 0ed56f0255a15dc0c24c239cd20f70729000bb3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Feb 2022 11:03:14 +0300 Subject: [PATCH 124/164] We don't use Uncrustify --- uncrustify.cfg | 252 ------------------------------------------------- 1 file changed, 252 deletions(-) delete mode 100644 uncrustify.cfg diff --git a/uncrustify.cfg b/uncrustify.cfg deleted file mode 100644 index 9fefc1270de..00000000000 --- a/uncrustify.cfg +++ /dev/null @@ -1,252 +0,0 @@ -# Configuration file for Uncrustify code formatter. -# https://github.com/uncrustify/uncrustify -# -# Created with https://cdanu.github.io/uncrustify_config_preview/index.html -# -# You may apply it for your code with: -# uncrustify -l CPP -c uncrustify.cfg -f filename.cpp -# -# This config is in beta: it doesn't implement our style guide perfectly. -# It's not recommended to apply it for existing code base. - -newlines = lf -input_tab_size = 4 -output_tab_size = 4 -string_replace_tab_chars = true -utf8_bom = remove -utf8_byte = true -utf8_force = true -sp_arith = force -sp_assign = force -sp_cpp_lambda_assign = remove -sp_cpp_lambda_paren = remove -sp_assign_default = force -sp_enum_assign = force -sp_enum_colon = force -sp_pp_concat = force -sp_pp_stringify = remove -sp_bool = force -sp_compare = force -sp_inside_paren = remove -sp_paren_paren = remove -sp_paren_brace = force -sp_before_ptr_star = force -sp_between_ptr_star = remove -sp_after_ptr_star = force -sp_after_ptr_star_qualifier = force -sp_after_ptr_star_func = force -sp_ptr_star_paren = force -sp_before_ptr_star_func = force -sp_before_byref = force -sp_before_unnamed_byref = force -sp_after_byref = force -sp_after_byref_func = force -sp_before_byref_func = force -sp_template_angle = force -sp_before_angle = remove -sp_inside_angle = remove -sp_angle_colon = force -sp_after_angle = force -sp_angle_paren = remove -sp_angle_paren_empty = remove -sp_angle_word = force -sp_angle_shift = remove -sp_permit_cpp11_shift = true -sp_before_sparen = force -sp_inside_sparen = remove -sp_after_sparen = force -sp_sparen_brace = force -sp_special_semi = force -sp_before_semi_for = remove -sp_before_semi_for_empty = remove -sp_after_semi = force -sp_after_semi_for_empty = remove -sp_before_square = remove -sp_before_squares = remove -sp_inside_square = remove -sp_after_comma = force -sp_before_ellipsis = remove -sp_after_class_colon = force -sp_before_class_colon = force -sp_after_constr_colon = force -sp_before_constr_colon = force -sp_after_operator = remove -sp_after_operator_sym = remove -sp_after_cast = remove -sp_inside_paren_cast = remove -sp_cpp_cast_paren = remove -sp_sizeof_paren = remove -sp_inside_braces_enum = force -sp_inside_braces_struct = force -sp_inside_braces = force -sp_inside_braces_empty = remove -sp_type_func = force -sp_func_proto_paren = remove -sp_func_proto_paren_empty = remove -sp_func_def_paren = remove -sp_func_def_paren_empty = remove -sp_inside_fparens = remove -sp_inside_fparen = remove -sp_inside_tparen = remove -sp_after_tparen_close = remove -sp_square_fparen = remove -sp_fparen_brace = force -sp_func_call_paren = remove -sp_func_class_paren = remove -sp_func_class_paren_empty = remove -sp_return_paren = force -sp_attribute_paren = remove -sp_defined_paren = remove -sp_throw_paren = force -sp_after_throw = force -sp_catch_paren = force -sp_macro = add -sp_macro_func = add -sp_else_brace = force -sp_brace_else = force -sp_brace_typedef = force -sp_catch_brace = force -sp_brace_catch = force -sp_try_brace = force -sp_word_brace = remove -sp_word_brace_ns = force -sp_before_dc = remove -sp_after_dc = remove -sp_cond_colon = force -sp_cond_colon_before = force -sp_cond_colon_after = force -sp_cond_question = force -sp_cond_question_before = force -sp_cond_question_after = force -sp_cond_ternary_short = remove -sp_cmt_cpp_start = force -sp_cmt_cpp_doxygen = true -sp_cmt_cpp_qttr = true -sp_endif_cmt = force -sp_after_new = force -sp_between_new_paren = remove -sp_after_newop_paren = force -sp_inside_newop_paren = remove -sp_before_tr_emb_cmt = force -indent_columns = 4 -indent_with_tabs = 0 -indent_namespace = false -indent_namespace_limit = 100 -indent_class = true -indent_ctor_init_leading = 1 -indent_shift = true -indent_func_call_param = true -indent_func_def_param = true -indent_func_proto_param = true -indent_func_class_param = true -indent_func_ctor_var_param = true -indent_template_param = true -indent_member = 4 -indent_switch_case = 4 -indent_switch_pp = false -indent_label = 0 -indent_access_spec = -4 -indent_paren_close = 2 -indent_paren_after_func_def = true -indent_paren_after_func_decl = true -indent_paren_after_func_call = true -indent_align_assign = false -indent_token_after_brace = false -indent_cpp_lambda_body = true -indent_ternary_operator = 1 -nl_assign_leave_one_liners = true -nl_class_leave_one_liners = true -nl_enum_leave_one_liners = true -nl_getset_leave_one_liners = true -nl_func_leave_one_liners = true -nl_cpp_lambda_leave_one_liners = true -nl_cpp_ldef_brace = add -nl_if_leave_one_liners = true -nl_start_of_file = remove -nl_end_of_file = force -nl_enum_brace = add -nl_struct_brace = add -nl_union_brace = add -nl_if_brace = add -nl_brace_else = add -nl_else_brace = add -nl_else_if = remove -nl_before_if_closing_paren = remove -nl_try_brace = add -nl_for_brace = add -nl_catch_brace = add -nl_brace_catch = add -nl_while_brace = add -nl_do_brace = add -nl_brace_while = remove -nl_switch_brace = add -nl_multi_line_define = true -nl_before_case = true -nl_after_case = true -nl_case_colon_brace = add -nl_namespace_brace = add -nl_template_class = add -nl_class_brace = add -nl_enum_own_lines = add -nl_func_scope_name = remove -nl_func_paren = remove -nl_func_def_paren = remove -nl_func_call_paren = remove -nl_func_call_paren_empty = remove -nl_func_decl_start_multi_line = true -nl_func_def_start_multi_line = true -nl_func_decl_args_multi_line = true -nl_func_def_args_multi_line = true -nl_func_decl_end = remove -nl_func_def_end = remove -nl_func_decl_empty = remove -nl_func_def_empty = remove -nl_func_call_empty = remove -nl_func_call_start_multi_line = true -nl_func_call_args_multi_line = true -nl_fdef_brace = add -nl_after_semicolon = true -nl_constr_colon = force -nl_split_if_one_liner = true -nl_split_for_one_liner = true -nl_split_while_one_liner = true -nl_max = 3 -nl_max_blank_in_func = 2 -nl_after_func_proto = 1 -nl_after_func_proto_group = 2 -nl_after_func_class_proto = 1 -nl_after_func_class_proto_group = 2 -nl_before_func_body_def = 1 -nl_before_func_body_proto = 1 -nl_after_func_body = 3 -nl_after_func_body_class = 3 -nl_after_func_body_one_liner = 1 -nl_after_multiline_comment = true -nl_after_struct = 3 -nl_before_class = 3 -nl_after_class = 3 -nl_before_access_spec = 2 -nl_after_access_spec = 1 -nl_after_try_catch_finally = 1 -eat_blanks_after_open_brace = true -eat_blanks_before_close_brace = true -nl_remove_extra_newlines = 1 -nl_after_return = true -pos_constr_comma = lead_break -pos_constr_colon = lead_force -code_width = 160 -ls_func_split_full = true -ls_code_width = true -align_left_shift = false -cmt_convert_tab_to_spaces = true -mod_full_brace_for = remove -mod_full_brace_if = remove -mod_full_brace_if_chain = true -mod_full_brace_while = remove -mod_paren_on_return = remove -mod_remove_extra_semicolon = true -mod_remove_empty_return = true -align_func_params = true -align_func_params_thresh = 140 -sp_inside_type_brace_init_lst = remove -nl_constr_init_args = add From dd4243ff1875f8f7d848e81e09d4ca41f8fe6349 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Feb 2022 11:05:01 +0300 Subject: [PATCH 125/164] Remove very old cruft --- .potato.yml | 27 --------------------------- docker-compose.yml | 15 --------------- 2 files changed, 42 deletions(-) delete mode 100644 .potato.yml delete mode 100644 docker-compose.yml diff --git a/.potato.yml b/.potato.yml deleted file mode 100644 index 7cb87c58bd1..00000000000 --- a/.potato.yml +++ /dev/null @@ -1,27 +0,0 @@ -# This is the configuration file with settings for Potato. -# Potato is an internal Yandex technology that allows us to sync internal [Yandex.Tracker](https://yandex.com/tracker/) and GitHub. - -# For all PRs where documentation is needed, just add a 'pr-feature' label and we will include it into documentation sprints. - -# The project name. -name: clickhouse -# Object handlers defines which handlers we use. -handlers: - # The handler for creating an Yandex.Tracker issue. - - name: issue-create - params: - triggers: - # The trigger for creating the Yandex.Tracker issue. When the specified event occurs, it transfers PR data to Yandex.Tracker. - github:pullRequest:labeled: - data: - # The Yandex.Tracker queue to create the issue in. Each issue in Tracker belongs to one of the project queues. - queue: CLICKHOUSEDOCS - # The issue title. - summary: '[Potato] Pull Request #{{pullRequest.number}}' - # The issue description. - description: > - {{pullRequest.description}} - - Ссылка на Pull Request: {{pullRequest.webUrl}} - # The condition for creating the Yandex.Tracker issue. - condition: eventPayload.labels.filter(label => ['pr-feature'].includes(label.name)).length diff --git a/docker-compose.yml b/docker-compose.yml deleted file mode 100644 index 3e3cfc38218..00000000000 --- a/docker-compose.yml +++ /dev/null @@ -1,15 +0,0 @@ -version: "2" - -services: - builder: - image: clickhouse/clickhouse-builder - build: docker/builder - client: - image: clickhouse/clickhouse-client - build: docker/client - command: ['--host', 'server'] - server: - image: clickhouse/clickhouse-server - build: docker/server - ports: - - 8123:8123 From 6fc3275553ba1d8d473aa8572cfbfb6220f618ba Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 12 Feb 2022 10:51:22 +0100 Subject: [PATCH 126/164] Add debug --- .../0_stateless/02049_clickhouse_local_merge_tree.expect | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect index ffa25b964db..0ed6f5e4821 100755 --- a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect +++ b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect @@ -1,5 +1,8 @@ #!/usr/bin/expect -f +# Temporarily export expect logs into /test_output -- should get along with other artefacts into ci +exp_internal -f /test_output/clickhouse_local_merge_tree_debug.log 0; + log_user 0 set timeout 20 match_max 100000 From 8e20cf9face8e210ae819ded42cc94e03c6f7200 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 12 Feb 2022 19:30:02 +0800 Subject: [PATCH 127/164] chmod bash shell --- tests/queries/0_stateless/02206_format_override.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02206_format_override.sh diff --git a/tests/queries/0_stateless/02206_format_override.sh b/tests/queries/0_stateless/02206_format_override.sh old mode 100644 new mode 100755 From cc55c8b98242911462016e4ec2756d1b4ce7d28e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 12 Feb 2022 15:01:20 +0300 Subject: [PATCH 128/164] Fix gtest_archive_reader_and_writer in case of !USE_MINIZIP Empty arrays/vectors are not supported by ValuesIn: /src/ch/clickhouse/src/IO/tests/gtest_archive_reader_and_writer.cpp:341:59: error: no matching function for call to 'ValuesIn' INSTANTIATE_TEST_SUITE_P(All, ArchiveReaderAndWriterTest, ::testing::ValuesIn(supported_archive_file_exts)); ^~~~~~~~~~~~~~~~~~~ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:459:43: note: expanded from macro 'INSTANTIATE_TEST_SUITE_P' return GTEST_EXPAND_(GTEST_GET_FIRST_(__VA_ARGS__, DUMMY_PARAM_)); \ ^~~~~~~~~~~ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:453:38: note: expanded from macro 'GTEST_GET_FIRST_' #define GTEST_GET_FIRST_(first, ...) first ^~~~~ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:452:28: note: expanded from macro 'GTEST_EXPAND_' #define GTEST_EXPAND_(arg) arg ^~~ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:301:29: note: candidate template ignored: substitution failure [with T = std::vector, N = 0]: zero-length arrays are not permitted in C++ internal::ParamGenerator ValuesIn(const T (&array)[N]) { ^ ~ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:306:58: note: candidate template ignored: substitution failure [with Container = std::vector [0]]: type 'std::vector [0]' cannot be used prior to '::' because it has no members internal::ParamGenerator ValuesIn( ~~~~~~~~~ ^ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:294:1: note: candidate function template not viable: requires 2 arguments, but 1 was provided ValuesIn(ForwardIterator begin, ForwardIterator end) { ^ /src/ch/clickhouse/src/IO/tests/gtest_archive_reader_and_writer.cpp:341:59: error: no matching function for call to 'ValuesIn' INSTANTIATE_TEST_SUITE_P(All, ArchiveReaderAndWriterTest, ::testing::ValuesIn(supported_archive_file_exts)); ^~~~~~~~~~~~~~~~~~~ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:468:32: note: expanded from macro 'INSTANTIATE_TEST_SUITE_P' auto t = std::make_tuple(__VA_ARGS__); \ ^~~~~~~~~~~ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:301:29: note: candidate template ignored: substitution failure [with T = std::vector, N = 0] internal::ParamGenerator ValuesIn(const T (&array)[N]) { ^ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:306:58: note: candidate template ignored: substitution failure [with Container = std::vector [0]]: type 'std::vector [0]' cannot be used prior to '::' because it has no members internal::ParamGenerator ValuesIn( ~~~~~~~~~ ^ /src/ch/clickhouse/contrib/googletest/googletest/include/gtest/gtest-param-test.h:294:1: note: candidate function template not viable: requires 2 arguments, but 1 was provided ValuesIn(ForwardIterator begin, ForwardIterator end) { ^ 2 errors generated. ninja: build stopped: subcommand failed. Signed-off-by: Azat Khuzhin --- src/IO/tests/gtest_archive_reader_and_writer.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index c6b012a9914..e1864415e1b 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -328,14 +328,16 @@ TEST_P(ArchiveReaderAndWriterTest, ArchiveNotExist) } +#if USE_MINIZIP + namespace { const char * supported_archive_file_exts[] = { -#if USE_MINIZIP ".zip", -#endif }; } INSTANTIATE_TEST_SUITE_P(All, ArchiveReaderAndWriterTest, ::testing::ValuesIn(supported_archive_file_exts)); + +#endif From e2c8ba9ab23e231299500e25ccdbf42082e15e73 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 12 Feb 2022 16:05:35 +0000 Subject: [PATCH 129/164] Added performance test --- src/Storages/StorageFile.cpp | 18 +++++++- tests/performance/file_table_function.xml | 54 +++++++++++++++++++++++ 2 files changed, 70 insertions(+), 2 deletions(-) create mode 100644 tests/performance/file_table_function.xml diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index ddc7717567a..9a2ec0789cd 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -199,18 +199,27 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user fs_table_path = user_files_absolute_path / fs_table_path; Strings paths; + /// Do not use fs::canonical or fs::weakly_canonical. /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. String path = fs::absolute(fs_table_path).lexically_normal(); /// Normalize path. - if (path.find_first_of("*?{") == std::string::npos) + + if (path.find(PartitionedSink::PARTITION_ID_WILDCARD) != std::string::npos) + { + paths.push_back(path); + } + else if (path.find_first_of("*?{") == std::string::npos) { std::error_code error; if (fs::exists(path)) total_bytes_to_read += fs::file_size(path, error); + paths.push_back(path); } else + { paths = listFilesWithRegexpMatching("/", path, total_bytes_to_read); + } for (const auto & cur_path : paths) checkCreationIsAllowed(context, user_files_absolute_path, cur_path); @@ -313,7 +322,11 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us is_db_table = false; paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); is_path_with_globs = paths.size() > 1; - path_for_partitioned_write = table_path_; + if (!paths.empty()) + path_for_partitioned_write = paths.front(); + else + path_for_partitioned_write = table_path_; + setStorageMetadata(args); } @@ -853,6 +866,7 @@ SinkToStoragePtr StorageFile::write( { if (path_for_partitioned_write.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty path for partitioned write"); + fs::create_directories(fs::path(path_for_partitioned_write).parent_path()); return std::make_shared( diff --git a/tests/performance/file_table_function.xml b/tests/performance/file_table_function.xml new file mode 100644 index 00000000000..143f2b5eb4d --- /dev/null +++ b/tests/performance/file_table_function.xml @@ -0,0 +1,54 @@ + + + + + format + + TabSeparated + TabSeparatedWithNames + TabSeparatedWithNamesAndTypes + CSV + CSVWithNames + Values + JSONEachRow + JSONCompactEachRow + JSONCompactEachRowWithNamesAndTypes + TSKV + RowBinary + Native + MsgPack + + + + partitions_count + + 5 + 50 + 500 + + + + + + INSERT INTO FUNCTION file('test_file', '{format}', 'key UInt64, value UInt64') + SELECT number, number FROM numbers(1000000) + + + + INSERT INTO FUNCTION file('test_file', '{format}', 'key UInt64, value1 UInt64, value2 UInt64, value3 UInt64, value4 UInt64, value5 UInt64') + SELECT number, number, number, number, number, number FROM numbers(1000000) + + + + INSERT INTO FUNCTION file('test_file_{{_partition_id}}', '{format}', 'partition_id UInt64, value UInt64') + PARTITION BY partition_id + SELECT (number % {partitions_count}) as partition_id, number FROM numbers(1000000) + + + + INSERT INTO FUNCTION file('test_file_{{_partition_id}}', '{format}', 'partition_id UInt64, value1 UInt64, value2 UInt64, value3 UInt64, value4 UInt64, value5 UInt64') + PARTITION BY partition_id + SELECT (number % {partitions_count}) as partition_id, number, number, number, number, number FROM numbers(1000000) + + + From fecf7f3d083d71604ddf001dcfaf0168ff00b6b7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 12 Feb 2022 18:24:09 +0100 Subject: [PATCH 130/164] May be fix test 01065_window_view_event_hop_watch_bounded.py --- base/base/LineReader.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/base/base/LineReader.cpp b/base/base/LineReader.cpp index 686d70f247d..f4e741a54e7 100644 --- a/base/base/LineReader.cpp +++ b/base/base/LineReader.cpp @@ -109,10 +109,10 @@ void LineReader::Suggest::addWords(Words && new_words) std::lock_guard lock(mutex); addNewWords(words, new_words, std::less{}); addNewWords(words_no_case, new_words_no_case, NoCaseCompare{}); - } - assert(std::is_sorted(words.begin(), words.end())); - assert(std::is_sorted(words_no_case.begin(), words_no_case.end(), NoCaseCompare{})); + assert(std::is_sorted(words.begin(), words.end())); + assert(std::is_sorted(words_no_case.begin(), words_no_case.end(), NoCaseCompare{})); + } } LineReader::LineReader(const String & history_file_path_, bool multiline_, Patterns extenders_, Patterns delimiters_) From 39049251b7e24d2aea420ac5b1efb9fe91a06354 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 12 Feb 2022 20:26:01 +0100 Subject: [PATCH 131/164] Fix disable_suggestion argument for clickhouse-local --- programs/local/LocalServer.cpp | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 70363c62cac..b1631aaa961 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -518,22 +518,17 @@ void LocalServer::processConfig() if (config().has("multiquery")) is_multiquery = true; - - load_suggestions = true; } else { - if (delayed_interactive) - { - load_suggestions = true; - } - need_render_progress = config().getBool("progress", false); echo_queries = config().hasOption("echo") || config().hasOption("verbose"); ignore_error = config().getBool("ignore-error", false); is_multiquery = true; } + print_stack_trace = config().getBool("stacktrace", false); + load_suggestions = (is_interactive || delayed_interactive) && !config().getBool("disable_suggestion", false); auto logging = (config().has("logger.console") || config().has("logger.level") From c83c5bc1f1ac65f218f51c6942f4288db548acb7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 12 Feb 2022 20:45:18 +0100 Subject: [PATCH 132/164] Remove debug --- .../0_stateless/02049_clickhouse_local_merge_tree.expect | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect index 0ed6f5e4821..ffa25b964db 100755 --- a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect +++ b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect @@ -1,8 +1,5 @@ #!/usr/bin/expect -f -# Temporarily export expect logs into /test_output -- should get along with other artefacts into ci -exp_internal -f /test_output/clickhouse_local_merge_tree_debug.log 0; - log_user 0 set timeout 20 match_max 100000 From 03f81c86855883450c0224b76867d6b8b0e2c5c7 Mon Sep 17 00:00:00 2001 From: Saad Ur Rahman Date: Sat, 12 Feb 2022 16:40:11 -0500 Subject: [PATCH 133/164] [CH-34239] HTTP User-Agent header (#34330) * [IO] HTTP User-Agent header Added HTTP User-Agent Header to HTTP requests. User-Agent: ClickHouse/VERSION_STRING Input header vectors could potentially contain User-Agent. If so, do not set another. * [Tests] HTTP User Agent Added stateless functional test 02205_HTTP_user_agent. Co-authored-by: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- src/IO/ReadWriteBufferFromHTTP.h | 13 ++ .../0_stateless/02205_HTTP_user_agent.python | 143 ++++++++++++++++++ .../02205_HTTP_user_agent.reference | 1 + .../0_stateless/02205_HTTP_user_agent.sh | 8 + 4 files changed, 165 insertions(+) create mode 100644 tests/queries/0_stateless/02205_HTTP_user_agent.python create mode 100644 tests/queries/0_stateless/02205_HTTP_user_agent.reference create mode 100755 tests/queries/0_stateless/02205_HTTP_user_agent.sh diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 4522f863db6..4e08a595484 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -291,6 +292,18 @@ namespace detail "0 < http_retry_initial_backoff_ms < settings.http_retry_max_backoff_ms (now 0 < {} < {})", settings.http_max_tries, settings.http_retry_initial_backoff_ms, settings.http_retry_max_backoff_ms); + // Configure User-Agent if it not already set. + const std::string user_agent = "User-Agent"; + auto iter = std::find_if(http_header_entries.begin(), http_header_entries.end(), [&user_agent](const HTTPHeaderEntry & entry) + { + return std::get<0>(entry) == user_agent; + }); + + if (iter == http_header_entries.end()) + { + http_header_entries.emplace_back(std::make_pair("User-Agent", fmt::format("ClickHouse/{}", VERSION_STRING))); + } + if (!delay_initialization) { initialize(); diff --git a/tests/queries/0_stateless/02205_HTTP_user_agent.python b/tests/queries/0_stateless/02205_HTTP_user_agent.python new file mode 100644 index 00000000000..8fb9cea0845 --- /dev/null +++ b/tests/queries/0_stateless/02205_HTTP_user_agent.python @@ -0,0 +1,143 @@ +#!/usr/bin/env python3 + +from http.server import SimpleHTTPRequestHandler,HTTPServer +import socket +import sys +import threading +import os +import traceback +import urllib.request +import subprocess + + +def is_ipv6(host): + try: + socket.inet_aton(host) + return False + except: + return True + +def get_local_port(host, ipv6): + if ipv6: + family = socket.AF_INET6 + else: + family = socket.AF_INET + + with socket.socket(family) as fd: + fd.bind((host, 0)) + return fd.getsockname()[1] + +CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', 'localhost') +CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') + +# Server returns this JSON response. +SERVER_JSON_RESPONSE = \ +'''{ + "login": "ClickHouse", + "id": 54801242, + "name": "ClickHouse", + "company": null +}''' + +EXPECTED_ANSWER = \ +'''{\\n\\t"login": "ClickHouse",\\n\\t"id": 54801242,\\n\\t"name": "ClickHouse",\\n\\t"company": null\\n}''' + +##################################################################################### +# This test starts an HTTP server and serves data to clickhouse url-engine based table. +# The objective of this test is to check the ClickHouse server provides a User-Agent +# with HTTP requests. +# In order for it to work ip+port of http server (given below) should be +# accessible from clickhouse server. +##################################################################################### + +# IP-address of this host accessible from the outside world. Get the first one +HTTP_SERVER_HOST = subprocess.check_output(['hostname', '-i']).decode('utf-8').strip().split()[0] +IS_IPV6 = is_ipv6(HTTP_SERVER_HOST) +HTTP_SERVER_PORT = get_local_port(HTTP_SERVER_HOST, IS_IPV6) + +# IP address and port of the HTTP server started from this script. +HTTP_SERVER_ADDRESS = (HTTP_SERVER_HOST, HTTP_SERVER_PORT) +if IS_IPV6: + HTTP_SERVER_URL_STR = 'http://' + f'[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}' + "/" +else: + HTTP_SERVER_URL_STR = 'http://' + f'{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + + +def get_ch_answer(query): + host = CLICKHOUSE_HOST + if IS_IPV6: + host = f'[{host}]' + + url = os.environ.get('CLICKHOUSE_URL', 'http://{host}:{port}'.format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP)) + return urllib.request.urlopen(url, data=query.encode()).read().decode() + +def check_answers(query, answer): + ch_answer = get_ch_answer(query) + if ch_answer.strip() != answer.strip(): + print("FAIL on query:", query, file=sys.stderr) + print("Expected answer:", answer, file=sys.stderr) + print("Fetched answer :", ch_answer, file=sys.stderr) + raise Exception("Fail on query") + +# Server with check for User-Agent headers. +class HttpProcessor(SimpleHTTPRequestHandler): + def _set_headers(self): + user_agent = self.headers.get('User-Agent') + if user_agent and user_agent.startswith('ClickHouse/'): + self.send_response(200) + else: + self.send_response(403) + + self.send_header('Content-Type', 'text/csv') + self.end_headers() + + def do_GET(self): + self._set_headers() + self.wfile.write(SERVER_JSON_RESPONSE.encode()) + + def log_message(self, format, *args): + return + +class HTTPServerV6(HTTPServer): + address_family = socket.AF_INET6 + +def start_server(requests_amount): + if IS_IPV6: + httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) + else: + httpd = HTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) + + def real_func(): + for i in range(requests_amount): + httpd.handle_request() + + t = threading.Thread(target=real_func) + return t + +##################################################################### +# Testing area. +##################################################################### + +def test_select(): + global HTTP_SERVER_URL_STR + query = 'SELECT * FROM url(\'{}\',\'JSONAsString\');'.format(HTTP_SERVER_URL_STR) + check_answers(query, EXPECTED_ANSWER) + +def main(): + t = start_server(1) + t.start() + test_select() + t.join() + print("PASSED") + +if __name__ == "__main__": + try: + main() + except Exception as ex: + exc_type, exc_value, exc_traceback = sys.exc_info() + traceback.print_tb(exc_traceback, file=sys.stderr) + print(ex, file=sys.stderr) + sys.stderr.flush() + + os._exit(1) + diff --git a/tests/queries/0_stateless/02205_HTTP_user_agent.reference b/tests/queries/0_stateless/02205_HTTP_user_agent.reference new file mode 100644 index 00000000000..53cdf1e9393 --- /dev/null +++ b/tests/queries/0_stateless/02205_HTTP_user_agent.reference @@ -0,0 +1 @@ +PASSED diff --git a/tests/queries/0_stateless/02205_HTTP_user_agent.sh b/tests/queries/0_stateless/02205_HTTP_user_agent.sh new file mode 100755 index 00000000000..b125e91ae85 --- /dev/null +++ b/tests/queries/0_stateless/02205_HTTP_user_agent.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +python3 "$CURDIR"/02205_HTTP_user_agent.python + From e02bd7d78503e9c2ccc3a2a1b684430078a1a6bb Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 12 Feb 2022 19:01:04 -0800 Subject: [PATCH 134/164] check and get columns in geoToH3 func --- src/Functions/geoToH3.cpp | 39 +++++++++++++++++++++++++++++++++------ 1 file changed, 33 insertions(+), 6 deletions(-) diff --git a/src/Functions/geoToH3.cpp b/src/Functions/geoToH3.cpp index 18951d1a03f..666130ed55e 100644 --- a/src/Functions/geoToH3.cpp +++ b/src/Functions/geoToH3.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int INCORRECT_DATA; + extern const int ILLEGAL_COLUMN; } namespace @@ -68,9 +69,35 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - const auto * col_lon = arguments[0].column.get(); - const auto * col_lat = arguments[1].column.get(); - const auto * col_res = arguments[2].column.get(); + const auto * col_lon = checkAndGetColumn(arguments[0].column.get()); + if (!col_lon) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal type {} of argument {} of function {}. Must be Float64.", + arguments[0].type->getName(), + 1, + getName()); + const auto & data_lon = col_lon->getData(); + + const auto * col_lat = checkAndGetColumn(arguments[1].column.get()); + if (!col_lat) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal type {} of argument {} of function {}. Must be Float64.", + arguments[1].type->getName(), + 2, + getName()); + const auto & data_lat = col_lat->getData(); + + const auto * col_res = checkAndGetColumn(arguments[2].column.get()); + if (!col_res) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal type {} of argument {} of function {}. Must be UInt8.", + arguments[2].type->getName(), + 3, + getName()); + const auto & data_res = col_res->getData(); auto dst = ColumnVector::create(); auto & dst_data = dst->getData(); @@ -78,9 +105,9 @@ public: for (size_t row = 0; row < input_rows_count; ++row) { - const double lon = col_lon->getFloat64(row); - const double lat = col_lat->getFloat64(row); - const UInt8 res = col_res->getUInt(row); + const double lon = data_lon[row]; + const double lat = data_lat[row]; + const UInt8 res = data_res[row]; LatLng coord; coord.lng = degsToRads(lon); From 4ccd1c32788f48dedf6b111a271bada1f0f8355f Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 13 Feb 2022 04:51:22 +0000 Subject: [PATCH 135/164] try fix data race in StorageLog --- src/Storages/StorageLog.cpp | 25 ++++++++++++++----------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 5f0bd240f64..5ba1514877a 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -732,8 +732,21 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & renameInMemory(new_table_id); } -void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) +static std::chrono::seconds getLockTimeout(ContextPtr context) { + const Settings & settings = context->getSettingsRef(); + Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); + if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) + lock_timeout = settings.max_execution_time.totalSeconds(); + return std::chrono::seconds{lock_timeout}; +} + +void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context, TableExclusiveLockHolder &) +{ + WriteLock lock{rwlock, getLockTimeout(context)}; + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + disk->clearDirectory(table_path); for (auto & data_file : data_files) @@ -750,16 +763,6 @@ void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr } -static std::chrono::seconds getLockTimeout(ContextPtr context) -{ - const Settings & settings = context->getSettingsRef(); - Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); - if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) - lock_timeout = settings.max_execution_time.totalSeconds(); - return std::chrono::seconds{lock_timeout}; -} - - Pipe StorageLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, From b50005e702da45d7028354ea008060113ef01527 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 12 Feb 2022 22:05:28 -0800 Subject: [PATCH 136/164] add check for max h3 resolution --- src/Functions/geoToH3.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Functions/geoToH3.cpp b/src/Functions/geoToH3.cpp index 666130ed55e..12f29422367 100644 --- a/src/Functions/geoToH3.cpp +++ b/src/Functions/geoToH3.cpp @@ -11,6 +11,7 @@ #include #include +#include #include @@ -109,6 +110,14 @@ public: const double lat = data_lat[row]; const UInt8 res = data_res[row]; + if (res > MAX_H3_RES) + throw Exception( + ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "The argument 'resolution' ({}) of function {} is out of bounds because the maximum resolution in H3 library is ", + toString(res), + getName(), + MAX_H3_RES); + LatLng coord; coord.lng = degsToRads(lon); coord.lat = degsToRads(lat); From 5f661fde484d5146168aa782451ab3e317658519 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 12 Feb 2022 22:58:48 -0800 Subject: [PATCH 137/164] fix tests --- tests/queries/0_stateless/00926_geo_to_h3.reference | 4 ++-- tests/queries/0_stateless/00926_geo_to_h3.sql | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00926_geo_to_h3.reference b/tests/queries/0_stateless/00926_geo_to_h3.reference index ad594f0e81f..074584ead16 100644 --- a/tests/queries/0_stateless/00926_geo_to_h3.reference +++ b/tests/queries/0_stateless/00926_geo_to_h3.reference @@ -4,12 +4,12 @@ 644325528491955313 644325528627451570 644325529094369568 -644325528491955313 +639821928864584823 644325528491955313 644325528491955313 644325528627451570 644325529094369568 -55.720762 37.598135 644325528491955313 +55.720762 37.598135 639821928864584823 55.720762 37.598135 644325528491955313 55.72076201 37.598135 644325528491955313 55.763241 37.660183 644325528627451570 diff --git a/tests/queries/0_stateless/00926_geo_to_h3.sql b/tests/queries/0_stateless/00926_geo_to_h3.sql index ed8e154fd9e..a86548d3555 100644 --- a/tests/queries/0_stateless/00926_geo_to_h3.sql +++ b/tests/queries/0_stateless/00926_geo_to_h3.sql @@ -11,9 +11,10 @@ INSERT INTO table1 VALUES(55.72076201, 37.59813500, 15); INSERT INTO table1 VALUES(55.72076200, 37.59813500, 14); select geoToH3(37.63098076, 55.77922738, 15); +select geoToH3(37.63098076, 55.77922738, 24); -- { serverError 69 } select geoToH3(lon, lat, resolution) from table1 order by lat, lon, resolution; -select geoToH3(lon, lat, 15) AS k from table1 order by lat, lon, k; -select lat, lon, geoToH3(lon, lat, 15) AS k from table1 order by lat, lon, k; +select geoToH3(lon, lat, resolution) AS k from table1 order by lat, lon, k; +select lat, lon, geoToH3(lon, lat, resolution) AS k from table1 order by lat, lon, k; select geoToH3(lon, lat, resolution) AS k, count(*) from table1 group by k order by k; DROP TABLE table1 From 4dadc7e15b3239fb66c31348f477c9e5c5ac58d2 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 12 Feb 2022 23:36:40 -0800 Subject: [PATCH 138/164] fix style check --- src/Functions/geoToH3.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/geoToH3.cpp b/src/Functions/geoToH3.cpp index 12f29422367..fb7301de776 100644 --- a/src/Functions/geoToH3.cpp +++ b/src/Functions/geoToH3.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int INCORRECT_DATA; extern const int ILLEGAL_COLUMN; + extern const int ARGUMENT_OUT_OF_BOUND; } namespace From 8e5ba9e7781f390de5e88f8c397062b0118c1d88 Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Thu, 10 Feb 2022 15:38:09 +0000 Subject: [PATCH 139/164] Avoid crash in case of GROUP BY LowCardinality(Nullable(String)) column and group_by_overflow_mode='any' --- src/Common/ColumnsHashing.h | 29 +++++++++++-------- .../02180_group_by_lowcardinality.reference | 10 +++++++ .../02180_group_by_lowcardinality.sql | 10 +++++++ 3 files changed, 37 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02180_group_by_lowcardinality.reference create mode 100644 tests/queries/0_stateless/02180_group_by_lowcardinality.sql diff --git a/src/Common/ColumnsHashing.h b/src/Common/ColumnsHashing.h index f32707798f7..8500a05795b 100644 --- a/src/Common/ColumnsHashing.h +++ b/src/Common/ColumnsHashing.h @@ -387,47 +387,52 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod } template - ALWAYS_INLINE FindResult findFromRow(Data & data, size_t row_, Arena & pool) + ALWAYS_INLINE FindResult findKey(Data & data, size_t row_, Arena & pool) { size_t row = getIndexAt(row_); if (is_nullable && row == 0) { if constexpr (has_mapped) - return FindResult(data.hasNullKeyData() ? &data.getNullKeyData() : nullptr, data.hasNullKeyData()); + return FindResult(data.hasNullKeyData() ? &data.getNullKeyData() : nullptr, data.hasNullKeyData(), 0); else - return FindResult(data.hasNullKeyData()); + return FindResult(data.hasNullKeyData(), 0); } if (visit_cache[row] != VisitValue::Empty) { if constexpr (has_mapped) - return FindResult(&mapped_cache[row], visit_cache[row] == VisitValue::Found); + return FindResult(&mapped_cache[row], visit_cache[row] == VisitValue::Found, 0); else - return FindResult(visit_cache[row] == VisitValue::Found); + return FindResult(visit_cache[row] == VisitValue::Found, 0); } auto key_holder = getKeyHolder(row_, pool); - typename Data::iterator it; + typename Data::LookupResult it; if (saved_hash) - it = data.find(*key_holder, saved_hash[row]); + it = data.find(keyHolderGetKey(key_holder), saved_hash[row]); else - it = data.find(*key_holder); + it = data.find(keyHolderGetKey(key_holder)); - bool found = it != data.end(); + bool found = it; visit_cache[row] = found ? VisitValue::Found : VisitValue::NotFound; if constexpr (has_mapped) { if (found) - mapped_cache[row] = it->second; + mapped_cache[row] = it->getMapped(); } + + size_t offset = 0; + + if constexpr (FindResult::has_offset) + offset = found ? data.offsetInternal(it) : 0; if constexpr (has_mapped) - return FindResult(&mapped_cache[row], found); + return FindResult(&mapped_cache[row], found, offset); else - return FindResult(found); + return FindResult(found, offset); } template diff --git a/tests/queries/0_stateless/02180_group_by_lowcardinality.reference b/tests/queries/0_stateless/02180_group_by_lowcardinality.reference new file mode 100644 index 00000000000..a7149390d1a --- /dev/null +++ b/tests/queries/0_stateless/02180_group_by_lowcardinality.reference @@ -0,0 +1,10 @@ +{"val":"1563.8","avg(toUInt32(val))":null} +{"val":"891.4","avg(toUInt32(val))":null} +{"val":"584.4","avg(toUInt32(val))":null} +{"val":"269","avg(toUInt32(val))":269} +{"val":"1233.4","avg(toUInt32(val))":null} +{"val":"1833","avg(toUInt32(val))":1833} +{"val":"1009.4","avg(toUInt32(val))":null} +{"val":"1178.6","avg(toUInt32(val))":null} +{"val":"372.6","avg(toUInt32(val))":null} +{"val":"232.4","avg(toUInt32(val))":null} diff --git a/tests/queries/0_stateless/02180_group_by_lowcardinality.sql b/tests/queries/0_stateless/02180_group_by_lowcardinality.sql new file mode 100644 index 00000000000..463753a624e --- /dev/null +++ b/tests/queries/0_stateless/02180_group_by_lowcardinality.sql @@ -0,0 +1,10 @@ +create table if not exists t_group_by_lowcardinality(p_date Date, val LowCardinality(Nullable(String))) +engine=MergeTree() partition by p_date order by tuple(); + +insert into t_group_by_lowcardinality select today() as p_date, toString(number/5) as val from numbers(10000); +insert into t_group_by_lowcardinality select today() as p_date, Null as val from numbers(100); + +select val, avg(toUInt32(val)) from t_group_by_lowcardinality group by val limit 10 settings max_threads=1, max_rows_to_group_by=100, group_by_overflow_mode='any' format JSONEachRow; + +drop table if exists t_group_by_lowcardinality; + From 4965586e7eadb2cda0c0867da51e09cd083a8263 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 13 Feb 2022 15:02:15 +0300 Subject: [PATCH 140/164] More gdb introspection on CI (#34517) * test/fuzzer: preserve core This may help to capture things like in [1]: Failed assertion: "bin->low_bits_full > bin->low_bits_empty" Received signal 6 Received signal Aborted (6) [1]: https://s3.amazonaws.com/clickhouse-test-reports/33057/19216f4c0ae0f72108c147f958a708b521ad27dc/fuzzer_astfuzzerdebug,actions//report.html Signed-off-by: Azat Khuzhin * test: do not run 'info locals' since 'backtrace full' includes it Signed-off-by: Azat Khuzhin * test: try capture backtrace from all threads Signed-off-by: Azat Khuzhin * test/stress: fix path for core artifacts Signed-off-by: Azat Khuzhin Fixes: #33389 * test/fuzzer: store core file in artifacts v2: fix report, because of undefined variable CORE_LINK v3: fix case when there is no core file Signed-off-by: Azat Khuzhin --- docker/test/fuzzer/run-fuzzer.sh | 15 ++++++++++++--- docker/test/stress/run.sh | 6 ++---- tests/ci/ast_fuzzer_check.py | 1 + 3 files changed, 15 insertions(+), 7 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 1ebaed752a6..e18c07bf2c1 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -185,15 +185,14 @@ handle SIGUSR2 nostop noprint pass handle SIG$RTMIN nostop noprint pass info signals continue +gcore backtrace full -info locals +thread apply all backtrace full info registers disassemble /s up -info locals disassemble /s up -info locals disassemble /s p \"done\" detach @@ -314,6 +313,11 @@ quit || echo "Fuzzer failed ($fuzzer_exit_code). See the logs." ; } \ | tail -1 > description.txt fi + + if test -f core.*; then + pigz core.* + mv core.*.gz core.gz + fi } case "$stage" in @@ -345,6 +349,10 @@ case "$stage" in time fuzz ;& "report") +CORE_LINK='' +if [ -f core.gz ]; then + CORE_LINK='core.gz' +fi cat > report.html < @@ -386,6 +394,7 @@ th { cursor: pointer; } fuzzer.log server.log main.log +${CORE_LINK}

diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 4387d16ea7c..e57dbc38ded 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -148,14 +148,12 @@ info signals continue gcore backtrace full -info locals +thread apply all backtrace full info registers disassemble /s up -info locals disassemble /s up -info locals disassemble /s p \"done\" detach @@ -269,5 +267,5 @@ clickhouse-local --structure "test String, res String" -q "SELECT 'failure', tes # Default filename is 'core.PROCESS_ID' for core in core.*; do pigz $core - mv $core.gz /output/ + mv $core.gz /test_output/ done diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 042e0e90459..319a6fc3fa5 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -98,6 +98,7 @@ if __name__ == "__main__": 'server.log': os.path.join(workspace_path, 'server.log'), 'fuzzer.log': os.path.join(workspace_path, 'fuzzer.log'), 'report.html': os.path.join(workspace_path, 'report.html'), + 'core.gz': os.path.join(workspace_path, 'core.gz'), } s3_helper = S3Helper('https://s3.amazonaws.com') From c4f1d7c40f15dadf66889826a1621b27aca9852c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 13 Feb 2022 14:38:43 +0100 Subject: [PATCH 141/164] Update url.md --- docs/en/engines/table-engines/special/url.md | 30 +++++++++----------- 1 file changed, 13 insertions(+), 17 deletions(-) diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 2d88aacf7e1..26d928085ce 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -9,28 +9,24 @@ Queries data to/from a remote HTTP/HTTPS server. This engine is similar to the [ Syntax: `URL(URL [,Format] [,CompressionMethod])` -- The `URL` parameter must conform to the structure of a Uniform Resource Locator. The specified URL must point to a server - that uses HTTP or HTTPS. This does not require any - additional headers for getting a response from the server. +- The `URL` parameter must conform to the structure of a Uniform Resource Locator. The specified URL must point to a server that uses HTTP or HTTPS. This does not require any additional headers for getting a response from the server. -- The `Format` must be one that ClickHouse can use in - `SELECT` queries and, if necessary, in `INSERTs`. For the full list of supported formats, see - [Formats](../../../interfaces/formats.md#formats). +- The `Format` must be one that ClickHouse can use in `SELECT` queries and, if necessary, in `INSERTs`. For the full list of supported formats, see [Formats](../../../interfaces/formats.md#formats). - `CompressionMethod` indicates that whether the HTTP body should be compressed. If the compression is enabled, the HTTP packets sent by the URL engine contain 'Content-Encoding' header to indicate which compression method is used. - To enable compression, please first make sure the remote HTTP endpoint indicated by the `URL` parameter supports corresponding compression algorithm. +To enable compression, please first make sure the remote HTTP endpoint indicated by the `URL` parameter supports corresponding compression algorithm. - The supported `CompressionMethod` should be one of following - - gzip or gz - - deflate - - brotli or br - - lzma or xz - - zstd or zst - - lz4 - - bz2 - - snappy - - none +The supported `CompressionMethod` should be one of following: +- gzip or gz +- deflate +- brotli or br +- lzma or xz +- zstd or zst +- lz4 +- bz2 +- snappy +- none ## Usage {#using-the-engine-in-the-clickhouse-server} From 9cb440e8d99bb32db3e5f2a208304a0bb4f3a18b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 13 Feb 2022 14:43:02 +0100 Subject: [PATCH 142/164] Update ColumnsHashing.h --- src/Common/ColumnsHashing.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ColumnsHashing.h b/src/Common/ColumnsHashing.h index 8500a05795b..fbd3e71f9b8 100644 --- a/src/Common/ColumnsHashing.h +++ b/src/Common/ColumnsHashing.h @@ -423,7 +423,7 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod if (found) mapped_cache[row] = it->getMapped(); } - + size_t offset = 0; if constexpr (FindResult::has_offset) From bce821ae52f7dd839cbfc9520991a5d990a065ee Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 13 Feb 2022 15:17:20 +0100 Subject: [PATCH 143/164] Update distributed.md --- docs/en/engines/table-engines/special/distributed.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index 36aeaf7c667..4d2454298f2 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -209,7 +209,7 @@ When querying a `Distributed` table, `SELECT` queries are sent to all shards and When the `max_parallel_replicas` option is enabled, query processing is parallelized across all replicas within a single shard. For more information, see the section [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). -To learn more about how distibuted `in` and `global in` queries are processed, refer to [this](../../../sql-reference/operators/in.md/#select-distributed-subqueries) documentation. +To learn more about how distibuted `in` and `global in` queries are processed, refer to [this](../../../sql-reference/operators/in.md#select-distributed-subqueries) documentation. ## Virtual Columns {#virtual-columns} From 10293acafbd40099fe4bdd8c490e72c00ab67d92 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 7 Feb 2022 21:34:40 +0000 Subject: [PATCH 144/164] Sort added equal items ranges randomization --- base/base/sort.h | 120 +++++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 111 insertions(+), 9 deletions(-) diff --git a/base/base/sort.h b/base/base/sort.h index 592a899a291..2576a05b632 100644 --- a/base/base/sort.h +++ b/base/base/sort.h @@ -2,6 +2,89 @@ #include +#ifdef NDEBUG + +#include + +/** Same as libcxx std::__debug_less. Just without dependency on private part of standard library. + * Check that Comparator induce strict weak ordering. + */ +template +class DebugLessComparator +{ +public: + constexpr DebugLessComparator(Comparator & cmp_) + : cmp(cmp_) + {} + + template + constexpr bool operator()(const LhsType & lhs, const RhsType & rhs) + { + bool lhs_less_than_rhs = cmp(lhs, rhs); + if (lhs_less_than_rhs) + assert(!cmp(rhs, lhs)); + + return lhs_less_than_rhs; + } + + template + constexpr bool operator()(LhsType & lhs, RhsType & rhs) + { + bool lhs_less_than_rhs = cmp(lhs, rhs); + if (lhs_less_than_rhs) + assert(!cmp(rhs, lhs)); + + return lhs_less_than_rhs; + } + +private: + Comparator & cmp; +}; + +template +using ComparatorWrapper = DebugLessComparator; + +template +void shuffleItemsInEqualRanges(RandomIt first, RandomIt last, Compare compare) +{ + static pcg64 rng(randomSeed()); + + bool equal_range_started = false; + RandomIt equal_range_start_it; + + while (true) + { + auto next = first + 1; + if (next == last) + break; + + bool first_equal_second = compare(*first, *next) == compare(*next, *first); + + if (equal_range_started && !first_equal_second) + { + std::shuffle(equal_range_start_it, next, rng); + equal_range_started = false; + } + else if (!equal_range_started && first_equal_second) + { + equal_range_started = true; + equal_range_start_it = first; + } + + ++first; + } + + if (equal_range_started) + std::shuffle(equal_range_start_it, last, rng); +} + +#else + +template +using ComparatorWrapper = Comparator; + +#endif + #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wold-style-cast" @@ -10,19 +93,33 @@ template void nth_element(RandomIt first, RandomIt nth, RandomIt last) { - ::miniselect::floyd_rivest_select(first, nth, last); -} + using value_type = typename std::iterator_traits::value_type; + using comparator = std::less; -template -void partial_sort(RandomIt first, RandomIt middle, RandomIt last) -{ - ::miniselect::floyd_rivest_partial_sort(first, middle, last); + comparator compare; + ComparatorWrapper compare_wrapper = compare; + + ::miniselect::floyd_rivest_select(first, nth, last, compare_wrapper); } template void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compare) { - ::miniselect::floyd_rivest_partial_sort(first, middle, last, compare); + ComparatorWrapper compare_wrapper = compare; + ::miniselect::floyd_rivest_partial_sort(first, middle, last, compare_wrapper); + +#ifdef NDEBUG + ::shuffleItemsInEqualRanges(first, middle, compare_wrapper); +#endif +} + + +template +void partial_sort(RandomIt first, RandomIt middle, RandomIt last) +{ + using value_type = typename std::iterator_traits::value_type; + using comparator = std::less; + ::partial_sort(first, middle, last, comparator()); } #pragma GCC diagnostic pop @@ -30,7 +127,12 @@ void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compar template void sort(RandomIt first, RandomIt last, Compare compare) { - ::pdqsort(first, last, compare); + ComparatorWrapper compare_wrapper = compare; + ::pdqsort(first, last, compare_wrapper); + +#ifdef NDEBUG + ::shuffleItemsInEqualRanges(first, last, compare_wrapper); +#endif } template @@ -38,5 +140,5 @@ void sort(RandomIt first, RandomIt last) { using value_type = typename std::iterator_traits::value_type; using comparator = std::less; - ::pdqsort(first, last, comparator()); + ::sort(first, last, comparator()); } From b536b27e15d7422c46c160abba5695bc3061f54f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 7 Feb 2022 22:14:26 +0000 Subject: [PATCH 145/164] Fixed tests --- base/base/sort.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/base/sort.h b/base/base/sort.h index 2576a05b632..bb142be3d39 100644 --- a/base/base/sort.h +++ b/base/base/sort.h @@ -5,6 +5,7 @@ #ifdef NDEBUG #include +#include /** Same as libcxx std::__debug_less. Just without dependency on private part of standard library. * Check that Comparator induce strict weak ordering. @@ -47,7 +48,7 @@ using ComparatorWrapper = DebugLessComparator; template void shuffleItemsInEqualRanges(RandomIt first, RandomIt last, Compare compare) { - static pcg64 rng(randomSeed()); + static pcg64 rng(getThreadId()); bool equal_range_started = false; RandomIt equal_range_start_it; @@ -113,7 +114,6 @@ void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compar #endif } - template void partial_sort(RandomIt first, RandomIt middle, RandomIt last) { From 42792dec48424923d29cd9b235f06c48394d4d74 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 8 Feb 2022 10:03:29 +0000 Subject: [PATCH 146/164] Simplified implementation --- base/base/sort.h | 58 ++++++++++++++++++++---------------------------- 1 file changed, 24 insertions(+), 34 deletions(-) diff --git a/base/base/sort.h b/base/base/sort.h index bb142be3d39..8ac7ca248bc 100644 --- a/base/base/sort.h +++ b/base/base/sort.h @@ -45,38 +45,12 @@ private: template using ComparatorWrapper = DebugLessComparator; -template -void shuffleItemsInEqualRanges(RandomIt first, RandomIt last, Compare compare) +template +void shuffle(RandomIt first, RandomIt last) { static pcg64 rng(getThreadId()); - bool equal_range_started = false; - RandomIt equal_range_start_it; - - while (true) - { - auto next = first + 1; - if (next == last) - break; - - bool first_equal_second = compare(*first, *next) == compare(*next, *first); - - if (equal_range_started && !first_equal_second) - { - std::shuffle(equal_range_start_it, next, rng); - equal_range_started = false; - } - else if (!equal_range_started && first_equal_second) - { - equal_range_started = true; - equal_range_start_it = first; - } - - ++first; - } - - if (equal_range_started) - std::shuffle(equal_range_start_it, last, rng); + std::shuffle(first, last, rng); } #else @@ -100,17 +74,32 @@ void nth_element(RandomIt first, RandomIt nth, RandomIt last) comparator compare; ComparatorWrapper compare_wrapper = compare; +#ifdef NDEBUG + ::shuffle(first, last); +#endif + ::miniselect::floyd_rivest_select(first, nth, last, compare_wrapper); + +#ifdef NDEBUG + ::shuffle(first, nth); + + if (nth != last) + ::shuffle(nth + 1, last); +#endif } template void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compare) { +#ifdef NDEBUG + ::shuffle(first, last); +#endif + ComparatorWrapper compare_wrapper = compare; ::miniselect::floyd_rivest_partial_sort(first, middle, last, compare_wrapper); #ifdef NDEBUG - ::shuffleItemsInEqualRanges(first, middle, compare_wrapper); + ::shuffle(middle, last); #endif } @@ -119,6 +108,7 @@ void partial_sort(RandomIt first, RandomIt middle, RandomIt last) { using value_type = typename std::iterator_traits::value_type; using comparator = std::less; + ::partial_sort(first, middle, last, comparator()); } @@ -127,12 +117,12 @@ void partial_sort(RandomIt first, RandomIt middle, RandomIt last) template void sort(RandomIt first, RandomIt last, Compare compare) { +#ifdef NDEBUG + ::shuffle(first, last); +#endif + ComparatorWrapper compare_wrapper = compare; ::pdqsort(first, last, compare_wrapper); - -#ifdef NDEBUG - ::shuffleItemsInEqualRanges(first, last, compare_wrapper); -#endif } template From 4a18e627ec13448185652725b3418bf1cb6d2339 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 8 Feb 2022 11:56:13 +0000 Subject: [PATCH 147/164] Updated implementation --- base/base/sort.h | 20 +++++--------------- 1 file changed, 5 insertions(+), 15 deletions(-) diff --git a/base/base/sort.h b/base/base/sort.h index 8ac7ca248bc..f38d4f88059 100644 --- a/base/base/sort.h +++ b/base/base/sort.h @@ -2,7 +2,7 @@ #include -#ifdef NDEBUG +// #ifdef NDEBUG #include #include @@ -53,12 +53,12 @@ void shuffle(RandomIt first, RandomIt last) std::shuffle(first, last, rng); } -#else +// #else -template -using ComparatorWrapper = Comparator; +// template +// using ComparatorWrapper = Comparator; -#endif +// #endif #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wold-style-cast" @@ -74,33 +74,25 @@ void nth_element(RandomIt first, RandomIt nth, RandomIt last) comparator compare; ComparatorWrapper compare_wrapper = compare; -#ifdef NDEBUG ::shuffle(first, last); -#endif ::miniselect::floyd_rivest_select(first, nth, last, compare_wrapper); -#ifdef NDEBUG ::shuffle(first, nth); if (nth != last) ::shuffle(nth + 1, last); -#endif } template void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compare) { -#ifdef NDEBUG ::shuffle(first, last); -#endif ComparatorWrapper compare_wrapper = compare; ::miniselect::floyd_rivest_partial_sort(first, middle, last, compare_wrapper); -#ifdef NDEBUG ::shuffle(middle, last); -#endif } template @@ -117,9 +109,7 @@ void partial_sort(RandomIt first, RandomIt middle, RandomIt last) template void sort(RandomIt first, RandomIt last, Compare compare) { -#ifdef NDEBUG ::shuffle(first, last); -#endif ComparatorWrapper compare_wrapper = compare; ::pdqsort(first, last, compare_wrapper); From 478ad91b7d7584686f0801961110c6d7ea9b2132 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 9 Feb 2022 11:23:20 +0000 Subject: [PATCH 148/164] Fixed tests --- .../00702_join_with_using_dups.sql | 2 +- .../01030_limit_by_with_ties_error.sh | 2 +- .../0_stateless/01031_semi_anti_join.sql | 8 +- .../01591_window_functions.reference | 158 +++++++++--------- .../0_stateless/01591_window_functions.sql | 22 +-- .../0_stateless/01670_neighbor_lc_bug.sql | 2 +- .../01780_column_sparse_full.reference | 10 +- .../0_stateless/01780_column_sparse_full.sql | 5 +- .../01881_join_on_conditions_merge.sql.j2 | 4 +- 9 files changed, 105 insertions(+), 108 deletions(-) diff --git a/tests/queries/0_stateless/00702_join_with_using_dups.sql b/tests/queries/0_stateless/00702_join_with_using_dups.sql index cf0c053a144..1499473ed11 100644 --- a/tests/queries/0_stateless/00702_join_with_using_dups.sql +++ b/tests/queries/0_stateless/00702_join_with_using_dups.sql @@ -23,7 +23,7 @@ select 'right subs'; select s.*, j.* from (select * from X) as s right join (select * from Y) as j using id order by s.id, j.id, s.x_name, j.y_name; select 'full'; -select X.*, Y.* from X full join Y using id order by X.id, Y.id; +select X.*, Y.* from X full join Y using id order by X.id, Y.id, X.x_name, Y.y_name; select 'full subs'; select s.*, j.* from (select * from X) as s full join (select * from Y) as j using id order by s.id, j.id, s.x_name, j.y_name; diff --git a/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh b/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh index 938eeff5b67..711a015f044 100755 --- a/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh +++ b/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh @@ -37,5 +37,5 @@ $CLICKHOUSE_CLIENT --query=""" ('upyachka', 'a'), ('test', 'b'), ('foo', 'c'), ('bar', 'd')) ORDER BY Payload LIMIT 1 BY Phrase - ) ORDER BY Payload, Payload + ) ORDER BY Payload, Phrase """ diff --git a/tests/queries/0_stateless/01031_semi_anti_join.sql b/tests/queries/0_stateless/01031_semi_anti_join.sql index 19ea219563a..388b3d2fe4c 100644 --- a/tests/queries/0_stateless/01031_semi_anti_join.sql +++ b/tests/queries/0_stateless/01031_semi_anti_join.sql @@ -10,16 +10,16 @@ INSERT INTO t2 (x, s) VALUES (2, 'b1'), (2, 'b2'), (4, 'b3'), (4, 'b4'), (4, 'b5 SET join_use_nulls = 0; SELECT 'semi left'; -SELECT t1.*, t2.* FROM t1 SEMI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; +SELECT t1.*, t2.* FROM t1 SEMI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x, t1.s, t2.s; SELECT 'semi right'; -SELECT t1.*, t2.* FROM t1 SEMI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; +SELECT t1.*, t2.* FROM t1 SEMI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x, t1.s, t2.s; SELECT 'anti left'; -SELECT t1.*, t2.* FROM t1 ANTI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; +SELECT t1.*, t2.* FROM t1 ANTI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x, t1.s, t2.s; SELECT 'anti right'; -SELECT t1.*, t2.* FROM t1 ANTI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; +SELECT t1.*, t2.* FROM t1 ANTI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x, t1.s, t2.s; DROP TABLE t1; DROP TABLE t2; diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 2f6077fceb3..8af2c4c6b25 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -39,7 +39,7 @@ select number, avg(number) over (order by number rows unbounded preceding) from 8 4 9 4.5 -- no order by -select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by value rows unbounded preceding) from numbers(10); +select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by number rows unbounded preceding) from numbers(10); 0 0 1 1 2 1 @@ -51,7 +51,7 @@ select number, quantileExact(number) over (partition by intDiv(number, 3) AS val 8 7 9 9 -- can add an alias after window spec -select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by value rows unbounded preceding) q from numbers(10); +select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by number rows unbounded preceding) q from numbers(10); 0 0 1 1 2 1 @@ -198,7 +198,7 @@ select sum(number) over w1, sum(number) over w2 from numbers(10) window w1 as (rows unbounded preceding), - w2 as (partition by intDiv(number, 3) as value order by value rows unbounded preceding) + w2 as (partition by intDiv(number, 3) as value order by number rows unbounded preceding) ; 0 0 1 1 @@ -214,7 +214,7 @@ window -- EXPLAIN test for this. select sum(number) over w1, - sum(number) over (partition by intDiv(number, 3) as value order by value rows unbounded preceding) + sum(number) over (partition by intDiv(number, 3) as value order by number rows unbounded preceding) from numbers(10) window w1 as (partition by intDiv(number, 3) rows unbounded preceding) @@ -240,118 +240,118 @@ select sum(number) over () from numbers(3); -- interesting corner cases. select number, intDiv(number, 3) p, mod(number, 2) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 5 ; -0 0 0 2 +0 0 0 1 1 0 1 3 2 0 0 2 -3 1 1 3 +3 1 1 2 4 1 0 1 5 1 1 3 -6 2 0 2 +6 2 0 1 7 2 1 3 8 2 0 2 -9 3 1 3 +9 3 1 2 10 3 0 1 11 3 1 3 -12 4 0 2 +12 4 0 1 13 4 1 3 14 4 0 2 -15 5 1 3 +15 5 1 2 16 5 0 1 17 5 1 3 -18 6 0 2 +18 6 0 1 19 6 1 3 20 6 0 2 -21 7 1 3 +21 7 1 2 22 7 0 1 23 7 1 3 -24 8 0 2 +24 8 0 1 25 8 1 3 26 8 0 2 -27 9 1 3 +27 9 1 2 28 9 0 1 29 9 1 3 30 10 0 1 select number, intDiv(number, 5) p, mod(number, 3) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 2 ; -0 0 0 2 -1 0 1 4 +0 0 0 1 +1 0 1 3 2 0 2 5 3 0 0 2 4 0 1 4 -5 1 2 5 -6 1 0 2 +5 1 2 4 +6 1 0 1 7 1 1 3 8 1 2 5 9 1 0 2 -10 2 1 3 -11 2 2 5 +10 2 1 2 +11 2 2 4 12 2 0 1 13 2 1 3 14 2 2 5 -15 3 0 2 -16 3 1 4 +15 3 0 1 +16 3 1 3 17 3 2 5 18 3 0 2 19 3 1 4 -20 4 2 5 -21 4 0 2 +20 4 2 4 +21 4 0 1 22 4 1 3 23 4 2 5 24 4 0 2 -25 5 1 3 -26 5 2 5 +25 5 1 2 +26 5 2 4 27 5 0 1 28 5 1 3 29 5 2 5 30 6 0 1 select number, intDiv(number, 5) p, mod(number, 2) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 3 ; -0 0 0 3 -1 0 1 5 -2 0 0 3 +0 0 0 1 +1 0 1 4 +2 0 0 2 3 0 1 5 4 0 0 3 -5 1 1 5 -6 1 0 2 -7 1 1 5 +5 1 1 3 +6 1 0 1 +7 1 1 4 8 1 0 2 9 1 1 5 -10 2 0 3 -11 2 1 5 -12 2 0 3 +10 2 0 1 +11 2 1 4 +12 2 0 2 13 2 1 5 14 2 0 3 -15 3 1 5 -16 3 0 2 -17 3 1 5 +15 3 1 3 +16 3 0 1 +17 3 1 4 18 3 0 2 19 3 1 5 -20 4 0 3 -21 4 1 5 -22 4 0 3 +20 4 0 1 +21 4 1 4 +22 4 0 2 23 4 1 5 24 4 0 3 -25 5 1 5 -26 5 0 2 -27 5 1 5 +25 5 1 3 +26 5 0 1 +27 5 1 4 28 5 0 2 29 5 1 5 30 6 0 1 select number, intDiv(number, 3) p, mod(number, 5) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 2 ; @@ -388,7 +388,7 @@ settings max_block_size = 2 30 10 0 1 select number, intDiv(number, 2) p, mod(number, 5) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 3 ; @@ -975,39 +975,39 @@ select number, p, o, row_number() over w from (select number, intDiv(number, 5) p, mod(number, 3) o from numbers(31) order by o, number) t -window w as (partition by p order by o) +window w as (partition by p order by o, number) order by p, o, number settings max_block_size = 2; -0 0 0 2 1 1 1 -3 0 0 2 1 1 2 -1 0 1 4 3 2 3 -4 0 1 4 3 2 4 -2 0 2 5 5 3 5 -6 1 0 2 1 1 1 -9 1 0 2 1 1 2 -7 1 1 3 3 2 3 -5 1 2 5 4 3 4 -8 1 2 5 4 3 5 +0 0 0 1 1 1 1 +3 0 0 2 2 2 2 +1 0 1 3 3 3 3 +4 0 1 4 4 4 4 +2 0 2 5 5 5 5 +6 1 0 1 1 1 1 +9 1 0 2 2 2 2 +7 1 1 3 3 3 3 +5 1 2 4 4 4 4 +8 1 2 5 5 5 5 12 2 0 1 1 1 1 -10 2 1 3 2 2 2 -13 2 1 3 2 2 3 -11 2 2 5 4 3 4 -14 2 2 5 4 3 5 -15 3 0 2 1 1 2 -18 3 0 2 1 1 1 -16 3 1 4 3 2 3 -19 3 1 4 3 2 4 -17 3 2 5 5 3 5 -21 4 0 2 1 1 1 -24 4 0 2 1 1 2 -22 4 1 3 3 2 3 -20 4 2 5 4 3 5 -23 4 2 5 4 3 4 +10 2 1 2 2 2 2 +13 2 1 3 3 3 3 +11 2 2 4 4 4 4 +14 2 2 5 5 5 5 +15 3 0 1 1 1 1 +18 3 0 2 2 2 2 +16 3 1 3 3 3 3 +19 3 1 4 4 4 4 +17 3 2 5 5 5 5 +21 4 0 1 1 1 1 +24 4 0 2 2 2 2 +22 4 1 3 3 3 3 +20 4 2 4 4 4 4 +23 4 2 5 5 5 5 27 5 0 1 1 1 1 -25 5 1 3 2 2 2 -28 5 1 3 2 2 3 -26 5 2 5 4 3 4 -29 5 2 5 4 3 5 +25 5 1 2 2 2 2 +28 5 1 3 3 3 3 +26 5 2 4 4 4 4 +29 5 2 5 5 5 5 30 6 0 1 1 1 1 -- our replacement for lag/lead select @@ -1153,7 +1153,7 @@ select count() over () where null; select number, count() over (w1 rows unbounded preceding) from numbers(10) window w0 as (partition by intDiv(number, 5) as p), - w1 as (w0 order by mod(number, 3) as o) + w1 as (w0 order by mod(number, 3) as o, number) order by p, o, number ; 0 1 diff --git a/tests/queries/0_stateless/01591_window_functions.sql b/tests/queries/0_stateless/01591_window_functions.sql index eb8c28de719..e1e0842ad89 100644 --- a/tests/queries/0_stateless/01591_window_functions.sql +++ b/tests/queries/0_stateless/01591_window_functions.sql @@ -13,10 +13,10 @@ select number, abs(number) over (partition by toString(intDiv(number, 3)) rows u select number, avg(number) over (order by number rows unbounded preceding) from numbers(10); -- no order by -select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by value rows unbounded preceding) from numbers(10); +select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by number rows unbounded preceding) from numbers(10); -- can add an alias after window spec -select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by value rows unbounded preceding) q from numbers(10); +select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by number rows unbounded preceding) q from numbers(10); -- can't reference it yet -- the window functions are calculated at the -- last stage of select, after all other functions. @@ -81,14 +81,14 @@ select sum(number) over w1, sum(number) over w2 from numbers(10) window w1 as (rows unbounded preceding), - w2 as (partition by intDiv(number, 3) as value order by value rows unbounded preceding) + w2 as (partition by intDiv(number, 3) as value order by number rows unbounded preceding) ; -- FIXME both functions should use the same window, but they don't. Add an -- EXPLAIN test for this. select sum(number) over w1, - sum(number) over (partition by intDiv(number, 3) as value order by value rows unbounded preceding) + sum(number) over (partition by intDiv(number, 3) as value order by number rows unbounded preceding) from numbers(10) window w1 as (partition by intDiv(number, 3) rows unbounded preceding) @@ -103,35 +103,35 @@ select sum(number) over () from numbers(3); -- interesting corner cases. select number, intDiv(number, 3) p, mod(number, 2) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 5 ; select number, intDiv(number, 5) p, mod(number, 3) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 2 ; select number, intDiv(number, 5) p, mod(number, 2) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 3 ; select number, intDiv(number, 3) p, mod(number, 5) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 2 ; select number, intDiv(number, 2) p, mod(number, 5) o, count(number) over w as c from numbers(31) -window w as (partition by p order by o range unbounded preceding) +window w as (partition by p order by o, number range unbounded preceding) order by number settings max_block_size = 3 ; @@ -349,7 +349,7 @@ select number, p, o, row_number() over w from (select number, intDiv(number, 5) p, mod(number, 3) o from numbers(31) order by o, number) t -window w as (partition by p order by o) +window w as (partition by p order by o, number) order by p, o, number settings max_block_size = 2; @@ -456,7 +456,7 @@ select count() over () where null; select number, count() over (w1 rows unbounded preceding) from numbers(10) window w0 as (partition by intDiv(number, 5) as p), - w1 as (w0 order by mod(number, 3) as o) + w1 as (w0 order by mod(number, 3) as o, number) order by p, o, number ; diff --git a/tests/queries/0_stateless/01670_neighbor_lc_bug.sql b/tests/queries/0_stateless/01670_neighbor_lc_bug.sql index 9de544b111d..f216befbb06 100644 --- a/tests/queries/0_stateless/01670_neighbor_lc_bug.sql +++ b/tests/queries/0_stateless/01670_neighbor_lc_bug.sql @@ -37,7 +37,7 @@ FROM ( SELECT * FROM neighbor_test - ORDER BY val_string ASC + ORDER BY val_string, rowNr ) ORDER BY rowNr, val_string, str_m1, str_p1, val_low, low_m1, low_p1 format PrettyCompact; diff --git a/tests/queries/0_stateless/01780_column_sparse_full.reference b/tests/queries/0_stateless/01780_column_sparse_full.reference index 4d2d0a58798..e725fb37cfc 100644 --- a/tests/queries/0_stateless/01780_column_sparse_full.reference +++ b/tests/queries/0_stateless/01780_column_sparse_full.reference @@ -35,11 +35,7 @@ all_2_2_0 u Default 4 [0,2,1,3] 8 ====== -0 0 0 -0 0 0 -0 0 0 -0 0 0 -1 1 1 +remote ====== 58413 57920 @@ -91,9 +87,9 @@ all_2_2_0 u Default ====== 0 0 0 0 0 0 -1 1 1 1 0 -2 2 2 +1 1 1 +2 0 ====== 0 0 0 0 0 0 diff --git a/tests/queries/0_stateless/01780_column_sparse_full.sql b/tests/queries/0_stateless/01780_column_sparse_full.sql index af6fde116d9..7cdacc43dd8 100644 --- a/tests/queries/0_stateless/01780_column_sparse_full.sql +++ b/tests/queries/0_stateless/01780_column_sparse_full.sql @@ -43,7 +43,8 @@ SELECT '======'; SELECT toUInt32(s) % 5 AS k, groupUniqArray(u % 4) FROM t_sparse_full WHERE s != '' GROUP BY k ORDER BY k; SELECT max(range(id % 10)[u]) FROM t_sparse_full; SELECT '======'; -SELECT id, u, s FROM remote('127.0.0.{1,2}', currentDatabase(), t_sparse_full) ORDER BY id LIMIT 5; +-- SELECT id, u, s FROM remote('127.0.0.{1,2}', currentDatabase(), t_sparse_full) ORDER BY id LIMIT 5; +SELECT 'remote'; SELECT '======'; SELECT sum(u) FROM t_sparse_full GROUP BY id % 3 AS k WITH TOTALS ORDER BY k; SELECT '======'; @@ -81,7 +82,7 @@ INNER JOIN t_sparse_full USING(u) ORDER BY id, u, s LIMIT 5; SELECT '======'; SELECT id, u, s FROM (SELECT number * 2 AS u FROM numbers(10)) AS t1 -FULL JOIN t_sparse_full USING(u) ORDER BY id LIMIT 5; +FULL JOIN t_sparse_full USING(u) ORDER BY id, u, s LIMIT 5; SELECT '======'; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index a51f4c856f3..1704fedb92b 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -24,9 +24,9 @@ SET join_algorithm = 'partial_merge'; SELECT '-- partial_merge --'; SELECT '--'; -SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2; +SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 ORDER BY t1.key, t1.key2; SELECT '--'; -SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2; +SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.key, t1.key2; SELECT '--'; SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2; From 05a6d8e118cf5c2bd5690b13b21d0836b200f40d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Feb 2022 10:24:46 +0000 Subject: [PATCH 149/164] Fixed tests --- tests/queries/0_stateless/00702_join_on_dups.sql | 10 +++++----- .../01525_select_with_offset_fetch_clause.reference | 2 +- .../01525_select_with_offset_fetch_clause.sql | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/00702_join_on_dups.sql b/tests/queries/0_stateless/00702_join_on_dups.sql index 48c80446f83..869d39308cc 100644 --- a/tests/queries/0_stateless/00702_join_on_dups.sql +++ b/tests/queries/0_stateless/00702_join_on_dups.sql @@ -33,7 +33,7 @@ select s.*, j.* from (select * from X) as s right join (select * from Y) as j on select 'full'; select X.*, Y.* from X full join Y on X.id = Y.id order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b; select 'full subs'; -select s.*, j.* from (select * from X) as s full join (select * from Y) as j on s.id = j.id order by s.id, s.x_a; +select s.*, j.* from (select * from X) as s full join (select * from Y) as j on s.id = j.id order by s.id, s.x_a, s.x_b, j.id, j.y_a, j.y_b; --select 'full expr'; --select X.*, Y.* from X full join Y on (X.id + 1) = (Y.id + 1) order by id; @@ -48,14 +48,14 @@ select 'self inner nullable vs not nullable 2'; select Y.*, s.* from Y inner join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by Y.id, Y.y_a, Y.y_b, s.id, s.y_a, s.y_b; select 'self left'; -select X.*, s.* from X left join (select * from X) as s on X.id = s.id order by X.id, X.x_a, s.x_a; +select X.*, s.* from X left join (select * from X) as s on X.id = s.id order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b; select 'self left nullable'; -select X.*, s.* from X left join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a; +select X.*, s.* from X left join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b; select 'self left nullable vs not nullable'; -select X.*, s.* from X left join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a; +select X.*, s.* from X left join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b; -- TODO: s.y_b == '' instead of NULL select 'self left nullable vs not nullable 2'; -select Y.*, s.* from Y left join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by Y.id, Y.y_a; +select Y.*, s.* from Y left join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by Y.id, Y.y_a, Y.y_b, s.id, s.y_a, s.y_b; select 'self right'; select X.*, s.* from X right join (select * from X) as s on X.id = s.id order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b; diff --git a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference index 19a1b0f2ec0..14e440bf205 100644 --- a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference +++ b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference @@ -3,7 +3,7 @@ 5 1 1 2 1 -3 3 +3 4 1 1 2 1 3 3 diff --git a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql index 3b6f77336fe..9689f57247e 100644 --- a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql +++ b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql @@ -3,6 +3,6 @@ SELECT number FROM numbers(10) ORDER BY number DESC OFFSET 2 ROWS FETCH NEXT 3 R DROP TABLE IF EXISTS test_fetch; CREATE TABLE test_fetch(a Int32, b Int32) Engine = Memory; INSERT INTO test_fetch VALUES(1, 1), (2, 1), (3, 4), (3, 3), (5, 4), (0, 6), (5, 7); -SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY; +SELECT * FROM (SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY) ORDER BY a, b; SELECT * FROM (SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS WITH TIES) ORDER BY a, b; DROP TABLE test_fetch; From 3bc70250ae7b198d5dbbae5129ca21ccfee62099 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Feb 2022 20:09:20 +0000 Subject: [PATCH 150/164] Fixed tests --- .../0_stateless/01525_select_with_offset_fetch_clause.reference | 2 +- .../0_stateless/01525_select_with_offset_fetch_clause.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference index 14e440bf205..19a1b0f2ec0 100644 --- a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference +++ b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.reference @@ -3,7 +3,7 @@ 5 1 1 2 1 -3 4 +3 3 1 1 2 1 3 3 diff --git a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql index 9689f57247e..d02a2af6666 100644 --- a/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql +++ b/tests/queries/0_stateless/01525_select_with_offset_fetch_clause.sql @@ -3,6 +3,6 @@ SELECT number FROM numbers(10) ORDER BY number DESC OFFSET 2 ROWS FETCH NEXT 3 R DROP TABLE IF EXISTS test_fetch; CREATE TABLE test_fetch(a Int32, b Int32) Engine = Memory; INSERT INTO test_fetch VALUES(1, 1), (2, 1), (3, 4), (3, 3), (5, 4), (0, 6), (5, 7); -SELECT * FROM (SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY) ORDER BY a, b; +SELECT * FROM (SELECT * FROM test_fetch ORDER BY a, b OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY) ORDER BY a, b; SELECT * FROM (SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS WITH TIES) ORDER BY a, b; DROP TABLE test_fetch; From 7cbf18ef592b5d92104987234c69d25cad0b1c3a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 11 Feb 2022 14:30:56 +0000 Subject: [PATCH 151/164] Fixed tests --- base/base/sort.h | 3 +-- .../00731_long_merge_tree_select_opened_files.sh | 3 +-- tests/queries/0_stateless/01086_odbc_roundtrip.reference | 8 -------- tests/queries/0_stateless/01086_odbc_roundtrip.sh | 4 ++-- tests/queries/0_stateless/01543_collate_in_tuple.sql | 6 +++--- tests/queries/0_stateless/01592_window_functions.sql | 2 +- .../0_stateless/01710_aggregate_projections.reference | 2 +- tests/queries/0_stateless/01710_aggregate_projections.sh | 2 +- 8 files changed, 10 insertions(+), 20 deletions(-) diff --git a/base/base/sort.h b/base/base/sort.h index f38d4f88059..283f11c31b1 100644 --- a/base/base/sort.h +++ b/base/base/sort.h @@ -48,8 +48,7 @@ using ComparatorWrapper = DebugLessComparator; template void shuffle(RandomIt first, RandomIt last) { - static pcg64 rng(getThreadId()); - + static thread_local pcg64 rng(getThreadId()); std::shuffle(first, last, rng); } diff --git a/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh b/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh index f9e33645527..02e02da4cd4 100755 --- a/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh +++ b/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh @@ -30,7 +30,6 @@ $CLICKHOUSE_CLIENT $settings -q "$touching_many_parts_query" &> /dev/null $CLICKHOUSE_CLIENT $settings -q "SYSTEM FLUSH LOGS" - -$CLICKHOUSE_CLIENT $settings -q "SELECT ProfileEvents['FileOpen'] FROM system.query_log WHERE query='$touching_many_parts_query' and current_database = currentDatabase() ORDER BY event_time DESC LIMIT 1;" +$CLICKHOUSE_CLIENT $settings -q "SELECT ProfileEvents['FileOpen'] as opened_files FROM system.query_log WHERE query='$touching_many_parts_query' and current_database = currentDatabase() ORDER BY event_time DESC, opened_files DESC LIMIT 1;" $CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS merge_tree_table;" diff --git a/tests/queries/0_stateless/01086_odbc_roundtrip.reference b/tests/queries/0_stateless/01086_odbc_roundtrip.reference index 2f84fe52456..e69de29bb2d 100644 --- a/tests/queries/0_stateless/01086_odbc_roundtrip.reference +++ b/tests/queries/0_stateless/01086_odbc_roundtrip.reference @@ -1,8 +0,0 @@ -1 -1 -1 0.1 a я -2 0.2 b ą -3 0.3 c d -1 0.1 a я -2 0.2 b ą -3 0.3 c d diff --git a/tests/queries/0_stateless/01086_odbc_roundtrip.sh b/tests/queries/0_stateless/01086_odbc_roundtrip.sh index 8e59bfd7f4d..705746032f8 100755 --- a/tests/queries/0_stateless/01086_odbc_roundtrip.sh +++ b/tests/queries/0_stateless/01086_odbc_roundtrip.sh @@ -17,7 +17,7 @@ ${CLICKHOUSE_CLIENT} --query "select count() > 1 as ok from (select * from odbc( ${CLICKHOUSE_CLIENT} --query "CREATE TABLE t (x UInt8, y Float32, z String) ENGINE = Memory" ${CLICKHOUSE_CLIENT} --query "INSERT INTO t VALUES (1,0.1,'a я'),(2,0.2,'b ą'),(3,0.3,'c d')" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM odbc('DSN={ClickHouse DSN (ANSI)}','$CLICKHOUSE_DATABASE','t') ORDER BY x" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM odbc('DSN={ClickHouse DSN (Unicode)}','$CLICKHOUSE_DATABASE','t') ORDER BY x" +${CLICKHOUSE_CLIENT} --query "SELECT x, y, z FROM odbc('DSN={ClickHouse DSN (ANSI)}','$CLICKHOUSE_DATABASE','t') ORDER BY x" +${CLICKHOUSE_CLIENT} --query "SELECT x, y, z FROM odbc('DSN={ClickHouse DSN (Unicode)}','$CLICKHOUSE_DATABASE','t') ORDER BY x" ${CLICKHOUSE_CLIENT} --query "DROP TABLE t" diff --git a/tests/queries/0_stateless/01543_collate_in_tuple.sql b/tests/queries/0_stateless/01543_collate_in_tuple.sql index 222f7762b32..e50b5e5223d 100644 --- a/tests/queries/0_stateless/01543_collate_in_tuple.sql +++ b/tests/queries/0_stateless/01543_collate_in_tuple.sql @@ -12,19 +12,19 @@ INSERT INTO collate_test1 VALUES (1, (1, 'Ё')), (1, (1, 'ё')), (1, (1, 'а')), INSERT INTO collate_test2 VALUES (1, (1, 'Ё')), (1, (1, 'ё')), (1, (1, 'а')), (2, (2, 'А')), (2, (1, 'я')), (2, (2, 'Я')), (1, (2, null)), (1, (3, 'я')), (1, (1, null)), (2, (2, null)); INSERT INTO collate_test3 VALUES (1, (1, (1, ['Ё']))), (1, (2, (1, ['ё']))), (1, (1, (2, ['а']))), (2, (1, (1, ['А']))), (2, (2, (1, ['я']))), (2, (1, (1, ['Я']))), (1, (2, (1, ['ё','а']))), (1, (1, (2, ['ё', 'я']))), (2, (1, (1, ['ё', 'а', 'а']))); -SELECT * FROM collate_test1 ORDER BY s COLLATE 'ru'; +SELECT * FROM collate_test1 ORDER BY s COLLATE 'ru', x; SELECT ''; SELECT * FROM collate_test1 ORDER BY x, s COLLATE 'ru'; SELECT ''; -SELECT * FROM collate_test2 ORDER BY s COLLATE 'ru'; +SELECT * FROM collate_test2 ORDER BY s COLLATE 'ru', x; SELECT ''; SELECT * FROM collate_test2 ORDER BY x, s COLLATE 'ru'; SELECT ''; -SELECT * FROM collate_test3 ORDER BY s COLLATE 'ru'; +SELECT * FROM collate_test3 ORDER BY s COLLATE 'ru', x; SELECT ''; SELECT * FROM collate_test3 ORDER BY x, s COLLATE 'ru'; diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index 1ef416aaa78..f0d173b1f20 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -36,7 +36,7 @@ SELECT price, rank() OVER (PARTITION BY group_name ORDER BY price) rank FROM products INNER JOIN product_groups USING (group_id) -order by group_name, rank, price; +order by group_name, rank, price, product_name; select '---- Q3 ----'; SELECT diff --git a/tests/queries/0_stateless/01710_aggregate_projections.reference b/tests/queries/0_stateless/01710_aggregate_projections.reference index f7cd6f29d9f..c99b082036d 100644 --- a/tests/queries/0_stateless/01710_aggregate_projections.reference +++ b/tests/queries/0_stateless/01710_aggregate_projections.reference @@ -13,8 +13,8 @@ 6 4 810 5 3 690 7 5 480 -3 2 450 7 4 450 +3 2 450 "rows_read": 18, 16 22080 15 19350 diff --git a/tests/queries/0_stateless/01710_aggregate_projections.sh b/tests/queries/0_stateless/01710_aggregate_projections.sh index a8b3e6bf99d..490584cc6fb 100755 --- a/tests/queries/0_stateless/01710_aggregate_projections.sh +++ b/tests/queries/0_stateless/01710_aggregate_projections.sh @@ -15,7 +15,7 @@ $CLICKHOUSE_CLIENT -q "select (x + y) * 2, sum(x - y) * 2 as s from test_agg_pro $CLICKHOUSE_CLIENT -q "select (x + y) * 2, sum(x - y) * 2 as s from test_agg_proj group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1 format JSON" | grep "rows_read" $CLICKHOUSE_CLIENT -q "select intDiv(x + y, 2), intDiv(x + y, 3), sum(x - y) as s from test_agg_proj group by intDiv(x + y, 2), intDiv(x + y, 3) order by s desc limit 5 settings allow_experimental_projection_optimization=1" -$CLICKHOUSE_CLIENT -q "select intDiv(x + y, 2), intDiv(x + y, 3), sum(x - y) as s from test_agg_proj group by intDiv(x + y, 2), intDiv(x + y, 3) order by s desc limit 5 settings allow_experimental_projection_optimization=1 format JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -q "select intDiv(x + y, 2) as v, intDiv(x + y, 3), sum(x - y) as s from test_agg_proj group by intDiv(x + y, 2), intDiv(x + y, 3) order by s desc, v limit 5 settings allow_experimental_projection_optimization=1 format JSON" | grep "rows_read" $CLICKHOUSE_CLIENT -q "select x + y + 1, argMax(x, y) * sum(x - y) as s from test_agg_proj group by x + y + 1 order by s desc limit 5 settings allow_experimental_projection_optimization=1" $CLICKHOUSE_CLIENT -q "select x + y + 1, argMax(x, y) * sum(x - y) as s from test_agg_proj group by x + y + 1 order by s desc limit 5 settings allow_experimental_projection_optimization=1 format JSON" | grep "rows_read" From 1d69737395724533094a09f205b3afd6de1c6bee Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 12 Feb 2022 19:16:57 +0000 Subject: [PATCH 152/164] Fixed tests --- tests/queries/0_stateless/01086_odbc_roundtrip.reference | 8 ++++++++ .../0_stateless/01710_aggregate_projections.reference | 2 +- tests/queries/0_stateless/01710_aggregate_projections.sh | 2 +- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01086_odbc_roundtrip.reference b/tests/queries/0_stateless/01086_odbc_roundtrip.reference index e69de29bb2d..2f84fe52456 100644 --- a/tests/queries/0_stateless/01086_odbc_roundtrip.reference +++ b/tests/queries/0_stateless/01086_odbc_roundtrip.reference @@ -0,0 +1,8 @@ +1 +1 +1 0.1 a я +2 0.2 b ą +3 0.3 c d +1 0.1 a я +2 0.2 b ą +3 0.3 c d diff --git a/tests/queries/0_stateless/01710_aggregate_projections.reference b/tests/queries/0_stateless/01710_aggregate_projections.reference index c99b082036d..f7cd6f29d9f 100644 --- a/tests/queries/0_stateless/01710_aggregate_projections.reference +++ b/tests/queries/0_stateless/01710_aggregate_projections.reference @@ -13,8 +13,8 @@ 6 4 810 5 3 690 7 5 480 -7 4 450 3 2 450 +7 4 450 "rows_read": 18, 16 22080 15 19350 diff --git a/tests/queries/0_stateless/01710_aggregate_projections.sh b/tests/queries/0_stateless/01710_aggregate_projections.sh index 490584cc6fb..561b8927579 100755 --- a/tests/queries/0_stateless/01710_aggregate_projections.sh +++ b/tests/queries/0_stateless/01710_aggregate_projections.sh @@ -14,7 +14,7 @@ $CLICKHOUSE_CLIENT -q "select x + y, sum(x - y) as s from test_agg_proj group by $CLICKHOUSE_CLIENT -q "select (x + y) * 2, sum(x - y) * 2 as s from test_agg_proj group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1" $CLICKHOUSE_CLIENT -q "select (x + y) * 2, sum(x - y) * 2 as s from test_agg_proj group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1 format JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT -q "select intDiv(x + y, 2), intDiv(x + y, 3), sum(x - y) as s from test_agg_proj group by intDiv(x + y, 2), intDiv(x + y, 3) order by s desc limit 5 settings allow_experimental_projection_optimization=1" +$CLICKHOUSE_CLIENT -q "select intDiv(x + y, 2) as v, intDiv(x + y, 3), sum(x - y) as s from test_agg_proj group by intDiv(x + y, 2), intDiv(x + y, 3) order by s desc, v limit 5 settings allow_experimental_projection_optimization=1" $CLICKHOUSE_CLIENT -q "select intDiv(x + y, 2) as v, intDiv(x + y, 3), sum(x - y) as s from test_agg_proj group by intDiv(x + y, 2), intDiv(x + y, 3) order by s desc, v limit 5 settings allow_experimental_projection_optimization=1 format JSON" | grep "rows_read" $CLICKHOUSE_CLIENT -q "select x + y + 1, argMax(x, y) * sum(x - y) as s from test_agg_proj group by x + y + 1 order by s desc limit 5 settings allow_experimental_projection_optimization=1" From e13a0bfb39c67514ad4b6ceb574c13852cf55ecf Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 13 Feb 2022 15:10:38 +0000 Subject: [PATCH 153/164] Enable randomization of sort only in debug --- base/base/sort.h | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/base/base/sort.h b/base/base/sort.h index 283f11c31b1..589469fffaa 100644 --- a/base/base/sort.h +++ b/base/base/sort.h @@ -2,7 +2,7 @@ #include -// #ifdef NDEBUG +#ifndef NDEBUG #include #include @@ -52,12 +52,12 @@ void shuffle(RandomIt first, RandomIt last) std::shuffle(first, last, rng); } -// #else +#else -// template -// using ComparatorWrapper = Comparator; +template +using ComparatorWrapper = Comparator; -// #endif +#endif #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wold-style-cast" @@ -73,25 +73,33 @@ void nth_element(RandomIt first, RandomIt nth, RandomIt last) comparator compare; ComparatorWrapper compare_wrapper = compare; +#ifndef NDEBUG ::shuffle(first, last); +#endif ::miniselect::floyd_rivest_select(first, nth, last, compare_wrapper); +#ifndef NDEBUG ::shuffle(first, nth); if (nth != last) ::shuffle(nth + 1, last); +#endif } template void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compare) { +#ifndef NDEBUG ::shuffle(first, last); +#endif ComparatorWrapper compare_wrapper = compare; ::miniselect::floyd_rivest_partial_sort(first, middle, last, compare_wrapper); +#ifndef NDEBUG ::shuffle(middle, last); +#endif } template @@ -108,7 +116,9 @@ void partial_sort(RandomIt first, RandomIt middle, RandomIt last) template void sort(RandomIt first, RandomIt last, Compare compare) { +#ifndef NDEBUG ::shuffle(first, last); +#endif ComparatorWrapper compare_wrapper = compare; ::pdqsort(first, last, compare_wrapper); From 0ae948866a1105569176e30b4e8667270ff9b656 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sun, 13 Feb 2022 01:04:07 +0100 Subject: [PATCH 154/164] Create symlink hadoop -> ./hadoop-2.7.0 --- .../test/integration/kerberized_hadoop/Dockerfile | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/docker/test/integration/kerberized_hadoop/Dockerfile b/docker/test/integration/kerberized_hadoop/Dockerfile index e42d115999a..592c3e36ef7 100644 --- a/docker/test/integration/kerberized_hadoop/Dockerfile +++ b/docker/test/integration/kerberized_hadoop/Dockerfile @@ -15,9 +15,10 @@ RUN curl -o krb5-libs-1.10.3-65.el6.x86_64.rpm ftp://ftp.pbone.net/mirror/vault. rm -fr *.rpm RUN cd /tmp && \ - curl http://archive.apache.org/dist/commons/daemon/source/commons-daemon-1.0.15-src.tar.gz -o commons-daemon-1.0.15-src.tar.gz && \ - tar xzf commons-daemon-1.0.15-src.tar.gz && \ - cd commons-daemon-1.0.15-src/src/native/unix && \ - ./configure && \ - make && \ - cp ./jsvc /usr/local/hadoop-2.7.0/sbin + curl http://archive.apache.org/dist/commons/daemon/source/commons-daemon-1.0.15-src.tar.gz -o commons-daemon-1.0.15-src.tar.gz && \ + tar xzf commons-daemon-1.0.15-src.tar.gz && \ + cd commons-daemon-1.0.15-src/src/native/unix && \ + ./configure && \ + make && \ + cp ./jsvc /usr/local/hadoop-2.7.0/sbin && \ + [ -e /usr/local/hadoop ] || ln -s ./hadoop-2.7.0 /usr/local/hadoop From 2de14d9e43fd3025ae1a9166e3a73c3cc8ad0258 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Sun, 13 Feb 2022 20:52:13 +0100 Subject: [PATCH 155/164] Documentation for insert_deduplication_token setting The setting was introduced in #32304 --- docs/en/operations/settings/settings.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 986d5eadd80..b02ce78c543 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1803,6 +1803,20 @@ If an INSERTed block is skipped due to deduplication in the source table, there At the same time, this behaviour “breaks” `INSERT` idempotency. If an `INSERT` into the main table was successful and `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won’t receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` allows for changing this behaviour. On retry, a materialized view will receive the repeat insert and will perform a deduplication check by itself, ignoring check result for the source table, and will insert rows lost because of the first failure. +## insert_deduplication_token {#insert_deduplication_token} + +The setting allows a user to provide own deduplication semantic in MergeTree/ReplicatedMergeTree +For example, by providing a unique value for the setting in each INSERT statement, +user can avoid the same inserted data being deduplicated + +Possilbe values: + +- Any string + +Default value: empty string (disabled) + +`insert_deduplication_token` is used for deduplication _only_ when not empty + ## max_network_bytes {#settings-max-network-bytes} Limits the data volume (in bytes) that is received or transmitted over the network when executing a query. This setting applies to every individual query. From 23546ab825707a8e2aabe912b3faafd098031d41 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sun, 13 Feb 2022 14:39:30 +0100 Subject: [PATCH 156/164] Use the current kerberized-haddop image --- .../runner/compose/docker_compose_kerberized_hdfs.yml | 2 +- docker/test/integration/runner/dockerd-entrypoint.sh | 1 + tests/ci/integration_test_check.py | 3 +++ tests/integration/ci-runner.py | 2 +- tests/integration/runner | 2 ++ .../test_storage_kerberized_hdfs/kerberos_image_config.sh | 1 + 6 files changed, 9 insertions(+), 2 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml index 88be3e45085..e1b4d393169 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml @@ -4,7 +4,7 @@ services: kerberizedhdfs1: cap_add: - DAC_READ_SEARCH - image: clickhouse/kerberized-hadoop + image: clickhouse/kerberized-hadoop:${DOCKER_KERBERIZED_HADOOP_TAG:-latest} hostname: kerberizedhdfs1 restart: always volumes: diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 8109ef7ae64..34414abc3f5 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -45,6 +45,7 @@ export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} +export DOCKER_KERBERIZED_HADOOP_TAG=${DOCKER_KERBERIZED_HADOOP_TAG:=latest} cd /ClickHouse/tests/integration exec "$@" diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index e87528dd528..786521db418 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -23,6 +23,8 @@ from rerun_helper import RerunHelper from tee_popen import TeePopen +# When update, update +# integration/ci-runner.py:ClickhouseIntegrationTestsRunner.get_images_names too IMAGES = [ "clickhouse/integration-tests-runner", "clickhouse/mysql-golang-client", @@ -32,6 +34,7 @@ IMAGES = [ "clickhouse/postgresql-java-client", "clickhouse/integration-test", "clickhouse/kerberos-kdc", + "clickhouse/kerberized-hadoop", "clickhouse/integration-helper", "clickhouse/dotnet-client", ] diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 40cb2c6fdd7..c1a2e6cf0b1 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -235,7 +235,7 @@ class ClickhouseIntegrationTestsRunner: "clickhouse/mysql-java-client", "clickhouse/mysql-js-client", "clickhouse/mysql-php-client", "clickhouse/postgresql-java-client", "clickhouse/integration-test", "clickhouse/kerberos-kdc", - "clickhouse/dotnet-client", + "clickhouse/kerberized-hadoop", "clickhouse/dotnet-client", "clickhouse/integration-helper", ] diff --git a/tests/integration/runner b/tests/integration/runner index 3687ca4068c..737eaeef683 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -238,6 +238,8 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) elif image == "clickhouse/integration-test": env_tags += "-e {}={} ".format("DOCKER_BASE_TAG", tag) + elif image == "clickhouse/kerberized-hadoop": + env_tags += "-e {}={} ".format("DOCKER_KERBERIZED_HADOOP_TAG", tag) elif image == "clickhouse/kerberos-kdc": env_tags += "-e {}={} ".format("DOCKER_KERBEROS_KDC_TAG", tag) else: diff --git a/tests/integration/test_storage_kerberized_hdfs/kerberos_image_config.sh b/tests/integration/test_storage_kerberized_hdfs/kerberos_image_config.sh index 0a746eb1a67..45fb93792e0 100644 --- a/tests/integration/test_storage_kerberized_hdfs/kerberos_image_config.sh +++ b/tests/integration/test_storage_kerberized_hdfs/kerberos_image_config.sh @@ -90,6 +90,7 @@ create_admin_user() { } create_keytabs() { + rm /tmp/keytab/*.keytab # kadmin.local -q "addprinc -randkey hdfs/kerberizedhdfs1.${DOMAIN_REALM}@${REALM}" From 9340aab154b6acc4d04e78d7cda7b8eed5d83d2f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sun, 13 Feb 2022 14:42:51 +0100 Subject: [PATCH 157/164] Clean kerberos keytab between running --- .../test_storage_kerberized_kafka/kerberos_image_config.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh index 723868ec68a..07437c42359 100644 --- a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh +++ b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh @@ -90,6 +90,7 @@ create_admin_user() { } create_keytabs() { + rm /tmp/keytab/*.keytab kadmin.local -q "addprinc -randkey zookeeper/kafka_kerberized_zookeeper@${REALM}" kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kafka_kerberized_zookeeper.keytab zookeeper/kafka_kerberized_zookeeper@${REALM}" From 90cdf5f9c8f753d552990a51f7f543a4fb49863b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Feb 2022 01:24:42 +0100 Subject: [PATCH 158/164] Postpone a bit nightly builds to the least loaded time --- .github/workflows/nightly.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 2602b9c28d5..a172947b2fc 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -6,7 +6,7 @@ env: "on": schedule: - - cron: '0 0 * * *' + - cron: '13 3 * * *' jobs: DockerHubPushAarch64: From 1e82b04de30b307482f4c4b0f96f8aa5d354425d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sun, 13 Feb 2022 23:54:18 +0100 Subject: [PATCH 159/164] Apply black formatter to ci-runner to trigger license/cla --- tests/integration/ci-runner.py | 531 +++++++++++++++++++++++---------- 1 file changed, 372 insertions(+), 159 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index c1a2e6cf0b1..8f228d91e9e 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -1,17 +1,16 @@ #!/usr/bin/env python3 -import logging -import subprocess -import os -import glob -import time -import shutil from collections import defaultdict -import random -import json import csv -# for crc32 -import zlib +import glob +import json +import logging +import os +import random +import shutil +import subprocess +import time +import zlib # for crc32 MAX_RETRY = 3 @@ -25,54 +24,62 @@ CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-library-bridge" TRIES_COUNT = 10 MAX_TIME_SECONDS = 3600 -MAX_TIME_IN_SANDBOX = 20 * 60 # 20 minutes -TASK_TIMEOUT = 8 * 60 * 60 # 8 hours +MAX_TIME_IN_SANDBOX = 20 * 60 # 20 minutes +TASK_TIMEOUT = 8 * 60 * 60 # 8 hours + def stringhash(s): - return zlib.crc32(s.encode('utf-8')) + return zlib.crc32(s.encode("utf-8")) + def get_tests_to_run(pr_info): result = set([]) - changed_files = pr_info['changed_files'] + changed_files = pr_info["changed_files"] if changed_files is None: return [] for fpath in changed_files: - if 'tests/integration/test_' in fpath: - logging.info('File %s changed and seems like integration test', fpath) - result.add(fpath.split('/')[2]) + if "tests/integration/test_" in fpath: + logging.info("File %s changed and seems like integration test", fpath) + result.add(fpath.split("/")[2]) return list(result) def filter_existing_tests(tests_to_run, repo_path): result = [] for relative_test_path in tests_to_run: - if os.path.exists(os.path.join(repo_path, 'tests/integration', relative_test_path)): + if os.path.exists( + os.path.join(repo_path, "tests/integration", relative_test_path) + ): result.append(relative_test_path) else: - logging.info("Skipping test %s, seems like it was removed", relative_test_path) + logging.info( + "Skipping test %s, seems like it was removed", relative_test_path + ) return result def _get_deselect_option(tests): - return ' '.join(['--deselect {}'.format(t) for t in tests]) + return " ".join([f"--deselect {t}" for t in tests]) + # https://stackoverflow.com/questions/312443/how-do-you-split-a-list-into-evenly-sized-chunks def chunks(lst, n): """Yield successive n-sized chunks from lst.""" for i in range(0, len(lst), n): - yield lst[i:i + n] + yield lst[i : i + n] + def get_counters(fname): counters = { - "ERROR": set([]), - "PASSED": set([]), - "FAILED": set([]), + "ERROR": set([]), + "PASSED": set([]), + "FAILED": set([]), "SKIPPED": set([]), } - with open(fname, 'r') as out: + with open(fname, "r") as out: for line in out: line = line.strip() # Example of log: @@ -81,10 +88,10 @@ def get_counters(fname): # [gw0] [ 7%] ERROR test_mysql_protocol/test.py::test_golang_client # # And only the line with test status should be matched - if not('.py::' in line and ' ' in line): + if not (".py::" in line and " " in line): continue - line_arr = line.strip().split(' ') + line_arr = line.strip().split(" ") if len(line_arr) < 2: logging.debug("Strange line %s", line) continue @@ -97,9 +104,9 @@ def get_counters(fname): if state in counters: counters[state].add(test_name) else: - # will skip lines line: - # 30.76s call test_host_ip_change/test.py::test_ip_change_drop_dns_cache - # 5.71s teardown test_host_ip_change/test.py::test_user_access_ip_change[node1] + # will skip lines like: + # 30.76s call test_host_ip_change/test.py::test_ip_drop_cache + # 5.71s teardown test_host_ip_change/test.py::test_ip_change[node1] # and similar logging.debug("Strange state in line %s", line) @@ -109,13 +116,13 @@ def get_counters(fname): def parse_test_times(fname): read = False description_output = [] - with open(fname, 'r') as out: + with open(fname, "r") as out: for line in out: - if read and '==' in line: + if read and "==" in line: break if read and line.strip(): description_output.append(line.strip()) - if 'slowest durations' in line: + if "slowest durations" in line: read = True return description_output @@ -123,10 +130,10 @@ def parse_test_times(fname): def get_test_times(output): result = defaultdict(float) for line in output: - if '.py' in line: - line_arr = line.strip().split(' ') + if ".py" in line: + line_arr = line.strip().split(" ") test_time = line_arr[0] - test_name = ' '.join([elem for elem in line_arr[2:] if elem]) + test_name = " ".join([elem for elem in line_arr[2:] if elem]) if test_name not in result: result[test_name] = 0.0 result[test_name] += float(test_time[:-1]) @@ -134,21 +141,28 @@ def get_test_times(output): def clear_ip_tables_and_restart_daemons(): - logging.info("Dump iptables after run %s", subprocess.check_output("sudo iptables -L", shell=True)) + logging.info( + "Dump iptables after run %s", + subprocess.check_output("sudo iptables -L", shell=True), + ) try: logging.info("Killing all alive docker containers") - subprocess.check_output("timeout -s 9 10m docker kill $(docker ps -q)", shell=True) + subprocess.check_output( + "timeout -s 9 10m docker kill $(docker ps -q)", shell=True + ) except subprocess.CalledProcessError as err: logging.info("docker kill excepted: " + str(err)) try: logging.info("Removing all docker containers") - subprocess.check_output("timeout -s 9 10m docker rm $(docker ps -a -q) --force", shell=True) + subprocess.check_output( + "timeout -s 9 10m docker rm $(docker ps -a -q) --force", shell=True + ) except subprocess.CalledProcessError as err: logging.info("docker rm excepted: " + str(err)) # don't restart docker if it's disabled - if os.environ.get("CLICKHOUSE_TESTS_RUNNER_RESTART_DOCKER", '1') == '1': + if os.environ.get("CLICKHOUSE_TESTS_RUNNER_RESTART_DOCKER", "1") == "1": try: logging.info("Stopping docker daemon") subprocess.check_output("service docker stop", shell=True) @@ -177,27 +191,33 @@ def clear_ip_tables_and_restart_daemons(): # when rules will be empty, it will raise exception subprocess.check_output("sudo iptables -D DOCKER-USER 1", shell=True) except subprocess.CalledProcessError as err: - logging.info("All iptables rules cleared, " + str(iptables_iter) + "iterations, last error: " + str(err)) + logging.info( + "All iptables rules cleared, " + + str(iptables_iter) + + "iterations, last error: " + + str(err) + ) class ClickhouseIntegrationTestsRunner: - def __init__(self, result_path, params): self.result_path = result_path self.params = params - self.image_versions = self.params['docker_images_with_versions'] - self.shuffle_groups = self.params['shuffle_test_groups'] - self.flaky_check = 'flaky check' in self.params['context_name'] + self.image_versions = self.params["docker_images_with_versions"] + self.shuffle_groups = self.params["shuffle_test_groups"] + self.flaky_check = "flaky check" in self.params["context_name"] # if use_tmpfs is not set we assume it to be true, otherwise check - self.use_tmpfs = 'use_tmpfs' not in self.params or self.params['use_tmpfs'] - self.disable_net_host = 'disable_net_host' in self.params and self.params['disable_net_host'] + self.use_tmpfs = "use_tmpfs" not in self.params or self.params["use_tmpfs"] + self.disable_net_host = ( + "disable_net_host" in self.params and self.params["disable_net_host"] + ) self.start_time = time.time() self.soft_deadline_time = self.start_time + (TASK_TIMEOUT - MAX_TIME_IN_SANDBOX) - if 'run_by_hash_total' in self.params: - self.run_by_hash_total = self.params['run_by_hash_total'] - self.run_by_hash_num = self.params['run_by_hash_num'] + if "run_by_hash_total" in self.params: + self.run_by_hash_total = self.params["run_by_hash_total"] + self.run_by_hash_num = self.params["run_by_hash_num"] else: self.run_by_hash_total = 0 self.run_by_hash_num = 0 @@ -206,7 +226,7 @@ class ClickhouseIntegrationTestsRunner: return self.result_path def base_path(self): - return os.path.join(str(self.result_path), '../') + return os.path.join(str(self.result_path), "../") def should_skip_tests(self): return [] @@ -214,8 +234,10 @@ class ClickhouseIntegrationTestsRunner: def get_image_with_version(self, name): if name in self.image_versions: return name + ":" + self.image_versions[name] - logging.warn("Cannot find image %s in params list %s", name, self.image_versions) - if ':' not in name: + logging.warn( + "Cannot find image %s in params list %s", name, self.image_versions + ) + if ":" not in name: return name + ":latest" return name @@ -223,31 +245,44 @@ class ClickhouseIntegrationTestsRunner: name = self.get_images_names()[0] if name in self.image_versions: return self.image_versions[name] - logging.warn("Cannot find image %s in params list %s", name, self.image_versions) - return 'latest' + logging.warn( + "Cannot find image %s in params list %s", name, self.image_versions + ) + return "latest" def shuffle_test_groups(self): return self.shuffle_groups != 0 @staticmethod def get_images_names(): - return ["clickhouse/integration-tests-runner", "clickhouse/mysql-golang-client", - "clickhouse/mysql-java-client", "clickhouse/mysql-js-client", - "clickhouse/mysql-php-client", "clickhouse/postgresql-java-client", - "clickhouse/integration-test", "clickhouse/kerberos-kdc", - "clickhouse/kerberized-hadoop", "clickhouse/dotnet-client", - "clickhouse/integration-helper", ] - + return [ + "clickhouse/dotnet-client", + "clickhouse/integration-helper", + "clickhouse/integration-test", + "clickhouse/integration-tests-runner", + "clickhouse/kerberized-hadoop", + "clickhouse/kerberos-kdc", + "clickhouse/mysql-golang-client", + "clickhouse/mysql-java-client", + "clickhouse/mysql-js-client", + "clickhouse/mysql-php-client", + "clickhouse/postgresql-java-client", + ] def _can_run_with(self, path, opt): - with open(path, 'r') as script: + with open(path, "r") as script: for line in script: if opt in line: return True return False def _install_clickhouse(self, debs_path): - for package in ('clickhouse-common-static_', 'clickhouse-server_', 'clickhouse-client', 'clickhouse-common-static-dbg_'): # order matters + for package in ( + "clickhouse-common-static_", + "clickhouse-server_", + "clickhouse-client", + "clickhouse-common-static-dbg_", + ): # order matters logging.info("Installing package %s", package) for f in os.listdir(debs_path): if package in f: @@ -255,10 +290,12 @@ class ClickhouseIntegrationTestsRunner: logging.info("Package found in %s", full_path) log_name = "install_" + f + ".log" log_path = os.path.join(str(self.path()), log_name) - with open(log_path, 'w') as log: + with open(log_path, "w") as log: cmd = "dpkg -x {} .".format(full_path) logging.info("Executing installation cmd %s", cmd) - retcode = subprocess.Popen(cmd, shell=True, stderr=log, stdout=log).wait() + retcode = subprocess.Popen( + cmd, shell=True, stderr=log, stdout=log + ).wait() if retcode == 0: logging.info("Installation of %s successfull", full_path) else: @@ -267,18 +304,35 @@ class ClickhouseIntegrationTestsRunner: else: raise Exception("Package with {} not found".format(package)) logging.info("Unstripping binary") - # logging.info("Unstring %s", subprocess.check_output("eu-unstrip /usr/bin/clickhouse {}".format(CLICKHOUSE_BINARY_PATH), shell=True)) + # logging.info( + # "Unstring %s", + # subprocess.check_output( + # "eu-unstrip /usr/bin/clickhouse {}".format(CLICKHOUSE_BINARY_PATH), + # shell=True, + # ), + # ) logging.info("All packages installed") os.chmod(CLICKHOUSE_BINARY_PATH, 0o777) os.chmod(CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH, 0o777) os.chmod(CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH, 0o777) - shutil.copy(CLICKHOUSE_BINARY_PATH, os.getenv("CLICKHOUSE_TESTS_SERVER_BIN_PATH")) - shutil.copy(CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH, os.getenv("CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH")) - shutil.copy(CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH, os.getenv("CLICKHOUSE_TESTS_LIBRARY_BRIDGE_BIN_PATH")) + shutil.copy( + CLICKHOUSE_BINARY_PATH, os.getenv("CLICKHOUSE_TESTS_SERVER_BIN_PATH") + ) + shutil.copy( + CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH, + os.getenv("CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH"), + ) + shutil.copy( + CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH, + os.getenv("CLICKHOUSE_TESTS_LIBRARY_BRIDGE_BIN_PATH"), + ) def _compress_logs(self, dir, relpaths, result_path): - subprocess.check_call("tar czf {} -C {} {}".format(result_path, dir, ' '.join(relpaths)), shell=True) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + "tar czf {} -C {} {}".format(result_path, dir, " ".join(relpaths)), + shell=True, + ) def _get_runner_opts(self): result = [] @@ -292,22 +346,40 @@ class ClickhouseIntegrationTestsRunner: image_cmd = self._get_runner_image_cmd(repo_path) out_file = "all_tests.txt" out_file_full = "all_tests_full.txt" - cmd = "cd {repo_path}/tests/integration && " \ - "timeout -s 9 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " \ - "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " \ + cmd = ( + "cd {repo_path}/tests/integration && " + "timeout -s 9 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " + "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " "| grep -v 'SKIPPED' | sort -u > {out_file}".format( - repo_path=repo_path, runner_opts=self._get_runner_opts(), image_cmd=image_cmd, out_file=out_file, out_file_full=out_file_full) + repo_path=repo_path, + runner_opts=self._get_runner_opts(), + image_cmd=image_cmd, + out_file=out_file, + out_file_full=out_file_full, + ) + ) logging.info("Getting all tests with cmd '%s'", cmd) - subprocess.check_call(cmd, shell=True) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + cmd, shell=True + ) - all_tests_file_path = "{repo_path}/tests/integration/{out_file}".format(repo_path=repo_path, out_file=out_file) - if not os.path.isfile(all_tests_file_path) or os.path.getsize(all_tests_file_path) == 0: - all_tests_full_file_path = "{repo_path}/tests/integration/{out_file}".format(repo_path=repo_path, out_file=out_file_full) + all_tests_file_path = "{repo_path}/tests/integration/{out_file}".format( + repo_path=repo_path, out_file=out_file + ) + if ( + not os.path.isfile(all_tests_file_path) + or os.path.getsize(all_tests_file_path) == 0 + ): + all_tests_full_file_path = ( + "{repo_path}/tests/integration/{out_file}".format( + repo_path=repo_path, out_file=out_file_full + ) + ) if os.path.isfile(all_tests_full_file_path): # log runner output logging.info("runner output:") - with open(all_tests_full_file_path, 'r') as all_tests_full_file: + with open(all_tests_full_file_path, "r") as all_tests_full_file: for line in all_tests_full_file: line = line.rstrip() if line: @@ -315,7 +387,11 @@ class ClickhouseIntegrationTestsRunner: else: logging.info("runner output '%s' is empty", all_tests_full_file_path) - raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(all_tests_file_path)) + raise Exception( + "There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format( + all_tests_file_path + ) + ) all_tests = [] with open(all_tests_file_path, "r") as all_tests_file: @@ -324,9 +400,18 @@ class ClickhouseIntegrationTestsRunner: return list(sorted(all_tests)) def _get_parallel_tests_skip_list(self, repo_path): - skip_list_file_path = "{}/tests/integration/parallel_skip.json".format(repo_path) - if not os.path.isfile(skip_list_file_path) or os.path.getsize(skip_list_file_path) == 0: - raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(skip_list_file_path)) + skip_list_file_path = "{}/tests/integration/parallel_skip.json".format( + repo_path + ) + if ( + not os.path.isfile(skip_list_file_path) + or os.path.getsize(skip_list_file_path) == 0 + ): + raise Exception( + "There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format( + skip_list_file_path + ) + ) skip_list_tests = [] with open(skip_list_file_path, "r") as skip_list_file: @@ -336,7 +421,7 @@ class ClickhouseIntegrationTestsRunner: def group_test_by_file(self, tests): result = {} for test in tests: - test_file = test.split('::')[0] + test_file = test.split("::")[0] if test_file not in result: result[test_file] = [] result[test_file].append(test) @@ -344,7 +429,10 @@ class ClickhouseIntegrationTestsRunner: def _update_counters(self, main_counters, current_counters): for test in current_counters["PASSED"]: - if test not in main_counters["PASSED"] and test not in main_counters["FLAKY"]: + if ( + test not in main_counters["PASSED"] + and test not in main_counters["FLAKY"] + ): is_flaky = False if test in main_counters["FAILED"]: main_counters["FAILED"].remove(test) @@ -369,45 +457,63 @@ class ClickhouseIntegrationTestsRunner: main_counters[state].append(test) def _get_runner_image_cmd(self, repo_path): - image_cmd = '' - if self._can_run_with(os.path.join(repo_path, "tests/integration", "runner"), '--docker-image-version'): + image_cmd = "" + if self._can_run_with( + os.path.join(repo_path, "tests/integration", "runner"), + "--docker-image-version", + ): for img in self.get_images_names(): if img == "clickhouse/integration-tests-runner": runner_version = self.get_single_image_version() - logging.info("Can run with custom docker image version %s", runner_version) - image_cmd += ' --docker-image-version={} '.format(runner_version) + logging.info( + "Can run with custom docker image version %s", runner_version + ) + image_cmd += " --docker-image-version={} ".format(runner_version) else: - if self._can_run_with(os.path.join(repo_path, "tests/integration", "runner"), '--docker-compose-images-tags'): - image_cmd += '--docker-compose-images-tags={} '.format(self.get_image_with_version(img)) + if self._can_run_with( + os.path.join(repo_path, "tests/integration", "runner"), + "--docker-compose-images-tags", + ): + image_cmd += "--docker-compose-images-tags={} ".format( + self.get_image_with_version(img) + ) else: - image_cmd = '' + image_cmd = "" logging.info("Cannot run with custom docker image version :(") return image_cmd def _find_test_data_dirs(self, repo_path, test_names): relpaths = {} for test_name in test_names: - if '/' in test_name: - test_dir = test_name[:test_name.find('/')] + if "/" in test_name: + test_dir = test_name[: test_name.find("/")] else: test_dir = test_name if os.path.isdir(os.path.join(repo_path, "tests/integration", test_dir)): - for name in os.listdir(os.path.join(repo_path, "tests/integration", test_dir)): + for name in os.listdir( + os.path.join(repo_path, "tests/integration", test_dir) + ): relpath = os.path.join(os.path.join(test_dir, name)) - mtime = os.path.getmtime(os.path.join(repo_path, "tests/integration", relpath)) + mtime = os.path.getmtime( + os.path.join(repo_path, "tests/integration", relpath) + ) relpaths[relpath] = mtime return relpaths def _get_test_data_dirs_difference(self, new_snapshot, old_snapshot): res = set() for path in new_snapshot: - if (not path in old_snapshot) or (old_snapshot[path] != new_snapshot[path]): + if (path not in old_snapshot) or (old_snapshot[path] != new_snapshot[path]): res.add(path) return res - def try_run_test_group(self, repo_path, test_group, tests_in_group, num_tries, num_workers): + def try_run_test_group( + self, repo_path, test_group, tests_in_group, num_tries, num_workers + ): try: - return self.run_test_group(repo_path, test_group, tests_in_group, num_tries, num_workers) + return self.run_test_group( + repo_path, test_group, tests_in_group, num_tries, num_workers + ) except Exception as e: logging.info("Failed to run {}:\n{}".format(str(test_group), str(e))) counters = { @@ -423,7 +529,9 @@ class ClickhouseIntegrationTestsRunner: tests_times[test] = 0 return counters, tests_times, [] - def run_test_group(self, repo_path, test_group, tests_in_group, num_tries, num_workers): + def run_test_group( + self, repo_path, test_group, tests_in_group, num_tries, num_workers + ): counters = { "ERROR": [], "PASSED": [], @@ -441,7 +549,7 @@ class ClickhouseIntegrationTestsRunner: return counters, tests_times, [] image_cmd = self._get_runner_image_cmd(repo_path) - test_group_str = test_group.replace('/', '_').replace('.', '_') + test_group_str = test_group.replace("/", "_").replace(".", "_") log_paths = [] test_data_dirs = {} @@ -453,8 +561,8 @@ class ClickhouseIntegrationTestsRunner: test_names = set([]) for test_name in tests_in_group: if test_name not in counters["PASSED"]: - if '[' in test_name: - test_names.add(test_name[:test_name.find('[')]) + if "[" in test_name: + test_names.add(test_name[: test_name.find("[")]) else: test_names.add(test_name) @@ -464,47 +572,83 @@ class ClickhouseIntegrationTestsRunner: info_basename = test_group_str + "_" + str(i) + ".nfo" info_path = os.path.join(repo_path, "tests/integration", info_basename) - test_cmd = ' '.join([test for test in sorted(test_names)]) - parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" + test_cmd = " ".join([test for test in sorted(test_names)]) + parallel_cmd = ( + " --parallel {} ".format(num_workers) if num_workers > 0 else "" + ) # -r -- show extra test summary: # -f -- (f)ailed # -E -- (E)rror # -p -- (p)assed # -s -- (s)kipped cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner {} {} -t {} {} '-rfEps --run-id={} --color=no --durations=0 {}' | tee {}".format( - repo_path, self._get_runner_opts(), image_cmd, test_cmd, parallel_cmd, i, _get_deselect_option(self.should_skip_tests()), info_path) + repo_path, + self._get_runner_opts(), + image_cmd, + test_cmd, + parallel_cmd, + i, + _get_deselect_option(self.should_skip_tests()), + info_path, + ) log_basename = test_group_str + "_" + str(i) + ".log" log_path = os.path.join(repo_path, "tests/integration", log_basename) - with open(log_path, 'w') as log: + with open(log_path, "w") as log: logging.info("Executing cmd: %s", cmd) - retcode = subprocess.Popen(cmd, shell=True, stderr=log, stdout=log).wait() + retcode = subprocess.Popen( + cmd, shell=True, stderr=log, stdout=log + ).wait() if retcode == 0: logging.info("Run %s group successfully", test_group) else: logging.info("Some tests failed") extra_logs_names = [log_basename] - log_result_path = os.path.join(str(self.path()), 'integration_run_' + log_basename) + log_result_path = os.path.join( + str(self.path()), "integration_run_" + log_basename + ) shutil.copy(log_path, log_result_path) log_paths.append(log_result_path) - for pytest_log_path in glob.glob(os.path.join(repo_path, "tests/integration/pytest*.log")): - new_name = test_group_str + "_" + str(i) + "_" + os.path.basename(pytest_log_path) - os.rename(pytest_log_path, os.path.join(repo_path, "tests/integration", new_name)) + for pytest_log_path in glob.glob( + os.path.join(repo_path, "tests/integration/pytest*.log") + ): + new_name = ( + test_group_str + + "_" + + str(i) + + "_" + + os.path.basename(pytest_log_path) + ) + os.rename( + pytest_log_path, + os.path.join(repo_path, "tests/integration", new_name), + ) extra_logs_names.append(new_name) dockerd_log_path = os.path.join(repo_path, "tests/integration/dockerd.log") if os.path.exists(dockerd_log_path): - new_name = test_group_str + "_" + str(i) + "_" + os.path.basename(dockerd_log_path) - os.rename(dockerd_log_path, os.path.join(repo_path, "tests/integration", new_name)) + new_name = ( + test_group_str + + "_" + + str(i) + + "_" + + os.path.basename(dockerd_log_path) + ) + os.rename( + dockerd_log_path, + os.path.join(repo_path, "tests/integration", new_name), + ) extra_logs_names.append(new_name) if os.path.exists(info_path): extra_logs_names.append(info_basename) new_counters = get_counters(info_path) for state, tests in new_counters.items(): - logging.info("Tests with %s state (%s): %s", state, len(tests), tests) + logging.info( + "Tests with %s state (%s): %s", state, len(tests), tests + ) times_lines = parse_test_times(info_path) new_tests_times = get_test_times(times_lines) self._update_counters(counters, new_counters) @@ -512,19 +656,35 @@ class ClickhouseIntegrationTestsRunner: tests_times[test_name] = test_time test_data_dirs_new = self._find_test_data_dirs(repo_path, test_names) - test_data_dirs_diff = self._get_test_data_dirs_difference(test_data_dirs_new, test_data_dirs) + test_data_dirs_diff = self._get_test_data_dirs_difference( + test_data_dirs_new, test_data_dirs + ) test_data_dirs = test_data_dirs_new if extra_logs_names or test_data_dirs_diff: - extras_result_path = os.path.join(str(self.path()), "integration_run_" + test_group_str + "_" + str(i) + ".tar.gz") - self._compress_logs(os.path.join(repo_path, "tests/integration"), extra_logs_names + list(test_data_dirs_diff), extras_result_path) + extras_result_path = os.path.join( + str(self.path()), + "integration_run_" + test_group_str + "_" + str(i) + ".tar.gz", + ) + self._compress_logs( + os.path.join(repo_path, "tests/integration"), + extra_logs_names + list(test_data_dirs_diff), + extras_result_path, + ) log_paths.append(extras_result_path) if len(counters["PASSED"]) + len(counters["FLAKY"]) == len(tests_in_group): logging.info("All tests from group %s passed", test_group) break - if len(counters["PASSED"]) + len(counters["FLAKY"]) >= 0 and len(counters["FAILED"]) == 0 and len(counters["ERROR"]) == 0: - logging.info("Seems like all tests passed but some of them are skipped or deselected. Ignoring them and finishing group.") + if ( + len(counters["PASSED"]) + len(counters["FLAKY"]) >= 0 + and len(counters["FAILED"]) == 0 + and len(counters["ERROR"]) == 0 + ): + logging.info( + "Seems like all tests passed but some of them are skipped or " + "deselected. Ignoring them and finishing group." + ) break else: # Mark all non tried tests as errors, with '::' in name @@ -532,26 +692,28 @@ class ClickhouseIntegrationTestsRunner: # we run whole test dirs like "test_odbc_interaction" and don't # want to mark them as error so we filter by '::'. for test in tests_in_group: - if (test not in counters["PASSED"] and - test not in counters["ERROR"] and - test not in counters["SKIPPED"] and - test not in counters["FAILED"] and - '::' in test): + if ( + test not in counters["PASSED"] + and test not in counters["ERROR"] + and test not in counters["SKIPPED"] + and test not in counters["FAILED"] + and "::" in test + ): counters["ERROR"].append(test) return counters, tests_times, log_paths def run_flaky_check(self, repo_path, build_path): - pr_info = self.params['pr_info'] + pr_info = self.params["pr_info"] # pytest swears, if we require to run some tests which was renamed or deleted tests_to_run = filter_existing_tests(get_tests_to_run(pr_info), repo_path) if not tests_to_run: logging.info("No tests to run found") - return 'success', 'Nothing to run', [('Nothing to run', 'OK')], '' + return "success", "Nothing to run", [("Nothing to run", "OK")], "" self._install_clickhouse(build_path) - logging.info("Found '%s' tests to run", ' '.join(tests_to_run)) + logging.info("Found '%s' tests to run", " ".join(tests_to_run)) result_state = "success" description_prefix = "No flaky tests: " start = time.time() @@ -561,17 +723,20 @@ class ClickhouseIntegrationTestsRunner: for i in range(TRIES_COUNT): final_retry += 1 logging.info("Running tests for the %s time", i) - counters, tests_times, log_paths = self.try_run_test_group(repo_path, "flaky", tests_to_run, 1, 1) + counters, tests_times, log_paths = self.try_run_test_group( + repo_path, "flaky", tests_to_run, 1, 1 + ) logs += log_paths if counters["FAILED"]: - logging.info("Found failed tests: %s", ' '.join(counters["FAILED"])) + logging.info("Found failed tests: %s", " ".join(counters["FAILED"])) description_prefix = "Flaky tests found: " result_state = "failure" break if counters["ERROR"]: description_prefix = "Flaky tests found: " - logging.info("Found error tests: %s", ' '.join(counters["ERROR"])) - # NOTE "error" result state will restart the whole test task, so we use "failure" here + logging.info("Found error tests: %s", " ".join(counters["ERROR"])) + # NOTE "error" result state will restart the whole test task, + # so we use "failure" here result_state = "failure" break assert len(counters["FLAKY"]) == 0 @@ -591,8 +756,20 @@ class ClickhouseIntegrationTestsRunner: text_state = "FAIL" else: text_state = state - test_result += [(c + ' (✕' + str(final_retry) + ')', text_state, "{:.2f}".format(tests_times[c])) for c in counters[state]] - status_text = description_prefix + ', '.join([str(n).lower().replace('failed', 'fail') + ': ' + str(len(c)) for n, c in counters.items()]) + test_result += [ + ( + c + " (✕" + str(final_retry) + ")", + text_state, + "{:.2f}".format(tests_times[c]), + ) + for c in counters[state] + ] + status_text = description_prefix + ", ".join( + [ + str(n).lower().replace("failed", "fail") + ": " + str(len(c)) + for n, c in counters.items() + ] + ) return result_state, status_text, test_result, logs @@ -601,7 +778,10 @@ class ClickhouseIntegrationTestsRunner: return self.run_flaky_check(repo_path, build_path) self._install_clickhouse(build_path) - logging.info("Dump iptables before run %s", subprocess.check_output("sudo iptables -L", shell=True)) + logging.info( + "Dump iptables before run %s", + subprocess.check_output("sudo iptables -L", shell=True), + ) all_tests = self._get_all_tests(repo_path) if self.run_by_hash_total != 0: @@ -613,18 +793,36 @@ class ClickhouseIntegrationTestsRunner: all_tests = all_filtered_by_hash_tests parallel_skip_tests = self._get_parallel_tests_skip_list(repo_path) - logging.info("Found %s tests first 3 %s", len(all_tests), ' '.join(all_tests[:3])) - filtered_sequential_tests = list(filter(lambda test: test in all_tests, parallel_skip_tests)) - filtered_parallel_tests = list(filter(lambda test: test not in parallel_skip_tests, all_tests)) - not_found_tests = list(filter(lambda test: test not in all_tests, parallel_skip_tests)) - logging.info("Found %s tests first 3 %s, parallel %s, other %s", len(all_tests), ' '.join(all_tests[:3]), len(filtered_parallel_tests), len(filtered_sequential_tests)) - logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3])) + logging.info( + "Found %s tests first 3 %s", len(all_tests), " ".join(all_tests[:3]) + ) + filtered_sequential_tests = list( + filter(lambda test: test in all_tests, parallel_skip_tests) + ) + filtered_parallel_tests = list( + filter(lambda test: test not in parallel_skip_tests, all_tests) + ) + not_found_tests = list( + filter(lambda test: test not in all_tests, parallel_skip_tests) + ) + logging.info( + "Found %s tests first 3 %s, parallel %s, other %s", + len(all_tests), + " ".join(all_tests[:3]), + len(filtered_parallel_tests), + len(filtered_sequential_tests), + ) + logging.info( + "Not found %s tests first 3 %s", + len(not_found_tests), + " ".join(not_found_tests[:3]), + ) grouped_tests = self.group_test_by_file(filtered_sequential_tests) i = 0 for par_group in chunks(filtered_parallel_tests, PARALLEL_GROUP_SIZE): - grouped_tests["parallel{}".format(i)] = par_group - i+=1 + grouped_tests[f"parallel{i}"] = par_group + i += 1 logging.info("Found %s tests groups", len(grouped_tests)) counters = { @@ -646,12 +844,18 @@ class ClickhouseIntegrationTestsRunner: for group, tests in items_to_run: logging.info("Running test group %s containing %s tests", group, len(tests)) - group_counters, group_test_times, log_paths = self.try_run_test_group(repo_path, group, tests, MAX_RETRY, NUM_WORKERS) + group_counters, group_test_times, log_paths = self.try_run_test_group( + repo_path, group, tests, MAX_RETRY, NUM_WORKERS + ) total_tests = 0 for counter, value in group_counters.items(): - logging.info("Tests from group %s stats, %s count %s", group, counter, len(value)) + logging.info( + "Tests from group %s stats, %s count %s", group, counter, len(value) + ) counters[counter] += value - logging.info("Totally have %s with status %s", len(counters[counter]), counter) + logging.info( + "Totally have %s with status %s", len(counters[counter]), counter + ) total_tests += len(counters[counter]) logging.info("Totally finished tests %s/%s", total_tests, len(all_tests)) @@ -664,7 +868,9 @@ class ClickhouseIntegrationTestsRunner: break if counters["FAILED"] or counters["ERROR"]: - logging.info("Overall status failure, because we have tests in FAILED or ERROR state") + logging.info( + "Overall status failure, because we have tests in FAILED or ERROR state" + ) result_state = "failure" else: logging.info("Overall success!") @@ -678,42 +884,49 @@ class ClickhouseIntegrationTestsRunner: text_state = "FAIL" else: text_state = state - test_result += [(c, text_state, "{:.2f}".format(tests_times[c]), tests_log_paths[c]) for c in counters[state]] + test_result += [ + (c, text_state, "{:.2f}".format(tests_times[c]), tests_log_paths[c]) + for c in counters[state] + ] - failed_sum = len(counters['FAILED']) + len(counters['ERROR']) - status_text = "fail: {}, passed: {}, flaky: {}".format(failed_sum, len(counters['PASSED']), len(counters['FLAKY'])) + failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) + status_text = "fail: {}, passed: {}, flaky: {}".format( + failed_sum, len(counters["PASSED"]), len(counters["FLAKY"]) + ) if self.soft_deadline_time < time.time(): status_text = "Timeout, " + status_text result_state = "failure" - counters['FLAKY'] = [] + counters["FLAKY"] = [] if not counters or sum(len(counter) for counter in counters.values()) == 0: status_text = "No tests found for some reason! It's a bug" result_state = "failure" - if '(memory)' in self.params['context_name']: + if "(memory)" in self.params["context_name"]: result_state = "success" return result_state, status_text, test_result, [] + def write_results(results_file, status_file, results, status): - with open(results_file, 'w') as f: - out = csv.writer(f, delimiter='\t') + with open(results_file, "w") as f: + out = csv.writer(f, delimiter="\t") out.writerows(results) - with open(status_file, 'w') as f: - out = csv.writer(f, delimiter='\t') + with open(status_file, "w") as f: + out = csv.writer(f, delimiter="\t") out.writerow(status) + if __name__ == "__main__": - logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') + logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s") repo_path = os.environ.get("CLICKHOUSE_TESTS_REPO_PATH") build_path = os.environ.get("CLICKHOUSE_TESTS_BUILD_PATH") result_path = os.environ.get("CLICKHOUSE_TESTS_RESULT_PATH") params_path = os.environ.get("CLICKHOUSE_TESTS_JSON_PARAMS_PATH") - params = json.loads(open(params_path, 'r').read()) + params = json.loads(open(params_path, "r").read()) runner = ClickhouseIntegrationTestsRunner(result_path, params) logging.info("Running tests") From edc35f12549c12dd011d8f825d05e42be2208ef3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 14 Feb 2022 15:21:31 +0300 Subject: [PATCH 160/164] Fix indentation in ARRAY JOIN formatting Signed-off-by: Azat Khuzhin --- src/Parsers/ASTTablesInSelectQuery.cpp | 2 ++ .../0_stateless/02220_array_join_format.reference | 11 +++++++++++ tests/queries/0_stateless/02220_array_join_format.sql | 1 + 3 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/02220_array_join_format.reference create mode 100644 tests/queries/0_stateless/02220_array_join_format.sql diff --git a/src/Parsers/ASTTablesInSelectQuery.cpp b/src/Parsers/ASTTablesInSelectQuery.cpp index 4680acc4c64..7435b22f7b7 100644 --- a/src/Parsers/ASTTablesInSelectQuery.cpp +++ b/src/Parsers/ASTTablesInSelectQuery.cpp @@ -247,10 +247,12 @@ void ASTTableJoin::formatImpl(const FormatSettings & settings, FormatState & sta void ASTArrayJoin::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); frame.expression_list_prepend_whitespace = true; settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws + << indent_str << (kind == Kind::Left ? "LEFT " : "") << "ARRAY JOIN" << (settings.hilite ? hilite_none : ""); settings.one_line diff --git a/tests/queries/0_stateless/02220_array_join_format.reference b/tests/queries/0_stateless/02220_array_join_format.reference new file mode 100644 index 00000000000..b1978acfbfa --- /dev/null +++ b/tests/queries/0_stateless/02220_array_join_format.reference @@ -0,0 +1,11 @@ +SELECT + range_, + point_ +FROM +( + SELECT + range(0, 10) AS range_, + point_ + FROM system.one + ARRAY JOIN range(0, 10) AS point_ +) diff --git a/tests/queries/0_stateless/02220_array_join_format.sql b/tests/queries/0_stateless/02220_array_join_format.sql new file mode 100644 index 00000000000..afea6855877 --- /dev/null +++ b/tests/queries/0_stateless/02220_array_join_format.sql @@ -0,0 +1 @@ +explain syntax select * from (select range(0, 10) range_, point_ from system.one array join range_ as point_); From c469fb36ae9b684b63638a3e890872a7a942a271 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 14 Feb 2022 12:59:53 +0000 Subject: [PATCH 161/164] Fixed tests --- .../queries/0_stateless/01780_column_sparse_full.reference | 6 +++++- tests/queries/0_stateless/01780_column_sparse_full.sql | 3 +-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01780_column_sparse_full.reference b/tests/queries/0_stateless/01780_column_sparse_full.reference index e725fb37cfc..dbe815800a9 100644 --- a/tests/queries/0_stateless/01780_column_sparse_full.reference +++ b/tests/queries/0_stateless/01780_column_sparse_full.reference @@ -35,7 +35,11 @@ all_2_2_0 u Default 4 [0,2,1,3] 8 ====== -remote +0 0 0 +0 0 0 +0 0 0 +0 0 0 +1 1 1 ====== 58413 57920 diff --git a/tests/queries/0_stateless/01780_column_sparse_full.sql b/tests/queries/0_stateless/01780_column_sparse_full.sql index 7cdacc43dd8..59128223dba 100644 --- a/tests/queries/0_stateless/01780_column_sparse_full.sql +++ b/tests/queries/0_stateless/01780_column_sparse_full.sql @@ -43,8 +43,7 @@ SELECT '======'; SELECT toUInt32(s) % 5 AS k, groupUniqArray(u % 4) FROM t_sparse_full WHERE s != '' GROUP BY k ORDER BY k; SELECT max(range(id % 10)[u]) FROM t_sparse_full; SELECT '======'; --- SELECT id, u, s FROM remote('127.0.0.{1,2}', currentDatabase(), t_sparse_full) ORDER BY id LIMIT 5; -SELECT 'remote'; +SELECT id, u, s FROM remote('127.0.0.{1,2}', currentDatabase(), t_sparse_full) ORDER BY id LIMIT 5; SELECT '======'; SELECT sum(u) FROM t_sparse_full GROUP BY id % 3 AS k WITH TOTALS ORDER BY k; SELECT '======'; From 79144f8b9166eb79cd0af0a61cfbaf18cf3fa7c4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 14 Feb 2022 17:26:48 +0300 Subject: [PATCH 162/164] Update 02211_shcema_inference_from_stdin.sh --- tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh index 9b98e970729..2b469797f89 100755 --- a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh +++ b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 02cb5a880fe5a94d081ad10219f2b6c0a6ed735e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 14 Feb 2022 15:22:07 +0000 Subject: [PATCH 163/164] Remove integration test --- .../__init__.py | 0 .../configs/global_overcommit_tracker.xml | 4 -- .../test_global_overcommit_tracker/test.py | 46 ------------------- 3 files changed, 50 deletions(-) delete mode 100644 tests/integration/test_global_overcommit_tracker/__init__.py delete mode 100644 tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml delete mode 100644 tests/integration/test_global_overcommit_tracker/test.py diff --git a/tests/integration/test_global_overcommit_tracker/__init__.py b/tests/integration/test_global_overcommit_tracker/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml b/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml deleted file mode 100644 index 590759bd15d..00000000000 --- a/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml +++ /dev/null @@ -1,4 +0,0 @@ - - 50000000 - 500 - \ No newline at end of file diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py deleted file mode 100644 index 229fcd6514b..00000000000 --- a/tests/integration/test_global_overcommit_tracker/test.py +++ /dev/null @@ -1,46 +0,0 @@ -import pytest - -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance('node', main_configs=['configs/global_overcommit_tracker.xml']) - -@pytest.fixture(scope='module', autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - -TEST_QUERY_A = 'SELECT number FROM numbers(10000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user=1' -TEST_QUERY_B = 'SELECT number FROM numbers(10000) GROUP BY number SETTINGS max_guaranteed_memory_usage_for_user=2' - -def test_overcommited_is_killed(): - node.query("CREATE USER A") - node.query("GRANT ALL ON *.* TO A") - node.query("CREATE USER B") - node.query("GRANT ALL ON *.* TO B") - - responses_A = list() - responses_B = list() - for _ in range(100): - responses_A.append(node.get_query_request(TEST_QUERY_A, user="A")) - responses_B.append(node.get_query_request(TEST_QUERY_B, user="B")) - - overcommited_killed = False - for response in responses_A: - _, err = response.get_answer_and_error() - if "MEMORY_LIMIT_EXCEEDED" in err: - overcommited_killed = True - finished = False - for response in responses_B: - _, err = response.get_answer_and_error() - if err == "": - finished = True - - assert overcommited_killed and finished, "no overcommited task was killed or all tasks are killed" - - node.query("DROP USER IF EXISTS A") - node.query("DROP USER IF EXISTS B") From 8a2ef6a6f779a3c68efb96bf16db6021ba7ee266 Mon Sep 17 00:00:00 2001 From: Brian Hunter Date: Mon, 14 Feb 2022 09:15:56 -0800 Subject: [PATCH 164/164] Add Gigasheet to adopters --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 5efa1b971bc..9c7fab7424d 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -67,6 +67,7 @@ toc_title: Adopters | Geniee | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) | | Genotek | Bioinformatics | Main product | — | — | [Video, August 2020](https://youtu.be/v3KyZbz9lEE) | | Gigapipe | Managed ClickHouse | Main product | — | — | [Official website](https://gigapipe.com/) | +| Gigasheet | Analytics | Main product | — | — | Direct Reference, February 2022| | Glaber | Monitoring | Main product | — | — | [Website](https://glaber.io/) | | GraphCDN | CDN | Traffic Analytics | — | — | [Blog Post in English, August 2021](https://altinity.com/blog/delivering-insight-on-graphql-apis-with-clickhouse-at-graphcdn/) | | Grouparoo | Data Warehouse Integrations | Main product | — | — | [Official Website, November 2021](https://www.grouparoo.com/integrations) |
Test nameTest statusDescription