From efdda9cc9bcb7609e1ab6a0f99b52ec891954bdc Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Thu, 1 Feb 2018 20:55:08 +0300 Subject: [PATCH 001/315] Introduced ThreadStatus. [#CLICKHOUSE-2910] --- .../AggregateFunctionCount.h | 3 + dbms/src/Common/ActionBlocker.h | 9 +- dbms/src/Common/MemoryTracker.cpp | 40 ++-- dbms/src/Common/MemoryTracker.h | 48 ++-- dbms/src/Common/ProfileEvents.cpp | 78 ++++-- dbms/src/Common/ProfileEvents.h | 66 +++++- dbms/src/Common/ThreadStatus.cpp | 223 ++++++++++++++++++ dbms/src/Common/ThreadStatus.h | 54 +++++ .../AsynchronousBlockInputStream.h | 12 +- dbms/src/DataStreams/BlockIO.h | 3 +- .../DataStreams/CountingBlockOutputStream.h | 4 +- .../IProfilingBlockInputStream.cpp | 2 +- .../DataStreams/IProfilingBlockInputStream.h | 6 +- ...regatedMemoryEfficientBlockInputStream.cpp | 23 +- ...ggregatedMemoryEfficientBlockInputStream.h | 3 +- .../src/DataStreams/ParallelInputsProcessor.h | 8 +- dbms/src/Interpreters/Aggregator.cpp | 37 +-- .../ClusterProxy/executeQuery.cpp | 2 +- dbms/src/Interpreters/Context.h | 8 +- dbms/src/Interpreters/ExternalLoader.cpp | 2 +- dbms/src/Interpreters/ProcessList.cpp | 132 ++++++++--- dbms/src/Interpreters/ProcessList.h | 91 +++---- .../Interpreters/ThreadPerformanceProfile.h | 23 ++ dbms/src/Interpreters/executeQuery.cpp | 4 +- dbms/src/Server/MetricsTransmitter.cpp | 2 +- .../DistributedBlockOutputStream.cpp | 10 +- .../MergeTree/BackgroundProcessingPool.cpp | 9 +- dbms/src/Storages/MergeTree/MergeList.cpp | 13 +- dbms/src/Storages/MergeTree/MergeList.h | 3 +- .../Storages/MergeTree/MergeTreeReader.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 2 +- dbms/src/Storages/StorageBuffer.cpp | 2 +- .../Storages/System/StorageSystemEvents.cpp | 2 +- 33 files changed, 705 insertions(+), 221 deletions(-) create mode 100644 dbms/src/Common/ThreadStatus.cpp create mode 100644 dbms/src/Common/ThreadStatus.h create mode 100644 dbms/src/Interpreters/ThreadPerformanceProfile.h diff --git a/dbms/src/AggregateFunctions/AggregateFunctionCount.h b/dbms/src/AggregateFunctions/AggregateFunctionCount.h index 5d39addff8b..c903b03e675 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionCount.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionCount.h @@ -42,6 +42,9 @@ public: void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { + if (!place || !rhs) + throw Exception("nullptr", ErrorCodes::LOGICAL_ERROR); + data(place).count += data(rhs).count; } diff --git a/dbms/src/Common/ActionBlocker.h b/dbms/src/Common/ActionBlocker.h index 72876a23e66..83cc6659d8f 100644 --- a/dbms/src/Common/ActionBlocker.h +++ b/dbms/src/Common/ActionBlocker.h @@ -7,10 +7,12 @@ namespace DB /// An atomic variable that is used to block and interrupt certain actions /// If it is not zero then actions related with it should be considered as interrupted -class ActionBlocker +template > +class ActionBlockerImpl { private: - mutable std::atomic counter{0}; + using ActionBlocker = ActionBlockerImpl; + mutable TCounter counter{0}; public: bool isCancelled() const { return counter > 0; } @@ -60,4 +62,7 @@ public: }; }; +using ActionBlocker = ActionBlockerImpl>; +using ActionBlockerSingleThread = ActionBlockerImpl; + } diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index 2c4d781056d..c1a7d520acb 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -3,9 +3,7 @@ #include #include #include -#include - -#include +#include namespace DB @@ -56,13 +54,16 @@ void MemoryTracker::logPeakMemoryUsage() const void MemoryTracker::alloc(Int64 size) { + if (blocker.isCancelled()) + return; + /** Using memory_order_relaxed means that if allocations are done simultaneously, * we allow exception about memory limit exceeded to be thrown only on next allocation. * So, we allow over-allocations. */ Int64 will_be = size + amount.fetch_add(size, std::memory_order_relaxed); - if (!next.load(std::memory_order_relaxed)) + if (!parent.load(std::memory_order_relaxed)) CurrentMetrics::add(metric, size); Int64 current_limit = limit.load(std::memory_order_relaxed); @@ -102,13 +103,16 @@ void MemoryTracker::alloc(Int64 size) if (will_be > peak.load(std::memory_order_relaxed)) /// Races doesn't matter. Could rewrite with CAS, but not worth. peak.store(will_be, std::memory_order_relaxed); - if (auto loaded_next = next.load(std::memory_order_relaxed)) + if (auto loaded_next = parent.load(std::memory_order_relaxed)) loaded_next->alloc(size); } void MemoryTracker::free(Int64 size) { + if (blocker.isCancelled()) + return; + Int64 new_amount = amount.fetch_sub(size, std::memory_order_relaxed) - size; /** Sometimes, query could free some data, that was allocated outside of query context. @@ -123,7 +127,7 @@ void MemoryTracker::free(Int64 size) size += new_amount; } - if (auto loaded_next = next.load(std::memory_order_relaxed)) + if (auto loaded_next = parent.load(std::memory_order_relaxed)) loaded_next->free(size); else CurrentMetrics::sub(metric, size); @@ -132,7 +136,7 @@ void MemoryTracker::free(Int64 size) void MemoryTracker::reset() { - if (!next.load(std::memory_order_relaxed)) + if (!parent.load(std::memory_order_relaxed)) CurrentMetrics::sub(metric, amount.load(std::memory_order_relaxed)); amount.store(0, std::memory_order_relaxed); @@ -149,29 +153,29 @@ void MemoryTracker::setOrRaiseLimit(Int64 value) ; } -#if __APPLE__ && __clang__ -__thread MemoryTracker * current_memory_tracker = nullptr; -#else -thread_local MemoryTracker * current_memory_tracker = nullptr; -#endif namespace CurrentMemoryTracker { void alloc(Int64 size) { - if (current_memory_tracker) - current_memory_tracker->alloc(size); + if (DB::current_thread) + DB::current_thread->memory_tracker.alloc(size); } void realloc(Int64 old_size, Int64 new_size) { - if (current_memory_tracker) - current_memory_tracker->alloc(new_size - old_size); + if (DB::current_thread) + DB::current_thread->memory_tracker.alloc(new_size - old_size); } void free(Int64 size) { - if (current_memory_tracker) - current_memory_tracker->free(size); + if (DB::current_thread) + DB::current_thread->memory_tracker.free(size); } } + +DB::ActionBlockerSingleThread::BlockHolder getCurrentMemoryTrackerBlocker() +{ + return (DB::current_thread) ? DB::current_thread->memory_tracker.blocker.cancel() : DB::ActionBlockerSingleThread::BlockHolder(nullptr); +} diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index e81eafbea05..573e829b6a3 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -26,7 +27,7 @@ class MemoryTracker /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. - std::atomic next {}; + std::atomic parent {}; /// You could specify custom metric to track memory usage. CurrentMetrics::Metric metric = CurrentMetrics::MemoryTracking; @@ -37,6 +38,7 @@ class MemoryTracker public: MemoryTracker() {} MemoryTracker(Int64 limit_) : limit(limit_) {} + MemoryTracker(MemoryTracker * parent_) : parent(parent_) {} ~MemoryTracker(); @@ -79,9 +81,15 @@ public: } /// next should be changed only once: from nullptr to some value. - void setNext(MemoryTracker * elem) + /// NOTE: It is not true in MergeListElement + void setParent(MemoryTracker * elem) { - next.store(elem, std::memory_order_relaxed); + parent.store(elem, std::memory_order_relaxed); + } + + MemoryTracker * getParent() + { + return parent.load(std::memory_order_relaxed); } /// The memory consumption could be shown in realtime via CurrentMetrics counter @@ -100,21 +108,13 @@ public: /// Prints info about peak memory consumption into log. void logPeakMemoryUsage() const; + + /// To be able to temporarily stop memory tracker + DB::ActionBlockerSingleThread blocker; }; -/** The MemoryTracker object is quite difficult to pass to all places where significant amounts of memory are allocated. - * Therefore, a thread-local pointer to used MemoryTracker is set, or nullptr if MemoryTracker does not need to be used. - * This pointer is set when memory consumption is monitored in current thread. - * So, you just need to pass it to all the threads that handle one request. - */ -#if __APPLE__ && __clang__ -extern __thread MemoryTracker * current_memory_tracker; -#else -extern thread_local MemoryTracker * current_memory_tracker; -#endif - -/// Convenience methods, that use current_memory_tracker if it is available. +/// Convenience methods, that use current thread's memory_tracker if it is available. namespace CurrentMemoryTracker { void alloc(Int64 size); @@ -123,20 +123,4 @@ namespace CurrentMemoryTracker } -#include - -struct TemporarilyDisableMemoryTracker : private boost::noncopyable -{ - MemoryTracker * memory_tracker; - - TemporarilyDisableMemoryTracker() - { - memory_tracker = current_memory_tracker; - current_memory_tracker = nullptr; - } - - ~TemporarilyDisableMemoryTracker() - { - current_memory_tracker = memory_tracker; - } -}; +DB::ActionBlockerSingleThread::BlockHolder getCurrentMemoryTrackerBlocker(); diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 71bf37c1a3a..9c1882c38de 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -1,4 +1,5 @@ #include +#include /// Available events. Add something here as you wish. @@ -132,30 +133,77 @@ M(RWLockAcquiredReadLocks) \ M(RWLockAcquiredWriteLocks) \ M(RWLockReadersWaitMilliseconds) \ - M(RWLockWritersWaitMilliseconds) + M(RWLockWritersWaitMilliseconds) \ + \ + M(RealTimeMicroseconds) \ + M(RusageUserTimeMicroseconds) \ + M(RusageSystemTimeMicroseconds) \ + M(RusagePageReclaims) \ + M(RusagePageVoluntaryContextSwitches) \ + M(RusagePageInvoluntaryContextSwitches) namespace ProfileEvents { - #define M(NAME) extern const Event NAME = __COUNTER__; + +#define M(NAME) extern const Event NAME = __COUNTER__; + APPLY_FOR_EVENTS(M) +#undef M +constexpr Event END = __COUNTER__; + +/// Global variable, initialized by zeros. +Counter global_counters_array[END] {}; +/// Initialize global counters statically +Counters global_counters(global_counters_array); + +const Event Counters::num_counters = END; + + +Counters::Counters(Level level, Counters * parent) + : parent(parent), level(level), + counters_holder(new Counter[num_counters] {}) +{ + counters = counters_holder.get(); +} + +void Counters::resetCounters() +{ + if (counters) + { + for (Event i = 0; i < num_counters; ++i) + counters[i].store(0, std::memory_order_relaxed); + } +} + +void Counters::reset() +{ + parent = nullptr; + resetCounters(); +} + +const char * getDescription(Event event) +{ + static const char * descriptions[] = + { + #define M(NAME) #NAME, APPLY_FOR_EVENTS(M) #undef M - constexpr Event END = __COUNTER__; + }; - std::atomic counters[END] {}; /// Global variable, initialized by zeros. + return descriptions[event]; +} - const char * getDescription(Event event) - { - static const char * descriptions[] = - { - #define M(NAME) #NAME, - APPLY_FOR_EVENTS(M) - #undef M - }; - return descriptions[event]; - } +Event end() { return END; } + + +void increment(Event event, Count amount) +{ + if (DB::current_thread) + DB::current_thread->performance_counters.increment(event, amount); + else + global_counters.increment(event, amount); +} - Event end() { return END; } } #undef APPLY_FOR_EVENTS diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index 0cb88f0ceed..302e5be1884 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -2,6 +2,7 @@ #include #include +#include /** Implements global counters for various events happening in the application @@ -14,19 +15,66 @@ namespace ProfileEvents /// Event identifier (index in array). using Event = size_t; using Count = size_t; + using Counter = std::atomic; + + enum class Level + { + Global = 0, + User, + Process, + Thread + }; + + struct Counters + { + Counter * counters = nullptr; + Counters * parent = nullptr; + const Level level = Level::Thread; + std::unique_ptr counters_holder; + + Counters() = default; + + Counters(Level level, Counters * parent = nullptr); + + /// Global level static initializer + Counters(Counter * allocated_counters) + : counters(allocated_counters), parent(nullptr), level(Level::Global) {} + + inline Counter & operator[] (Event event) + { + return counters[event]; + } + + inline void increment(Event event, Count amount = 1) + { + Counters * current = this; + do + { + current->counters[event].fetch_add(amount, std::memory_order_relaxed); + current = current->parent; + } while (current != nullptr); + } + + /// Reset metrics and parent + void reset(); + + /// Reset metrics + void resetCounters(); + + static const Event num_counters; + }; + + + /// Counters - how many times each event happened. + extern Counters global_counters; + + + /// Increment a counter for event. Thread-safe. + void increment(Event event, Count amount = 1); /// Get text description of event by identifier. Returns statically allocated string. const char * getDescription(Event event); - /// Counters - how many times each event happened. - extern std::atomic counters[]; - - /// Increment a counter for event. Thread-safe. - inline void increment(Event event, Count amount = 1) - { - counters[event].fetch_add(amount, std::memory_order_relaxed); - } - /// Get index just after last event identifier. Event end(); } diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp new file mode 100644 index 00000000000..52e0a29f3c9 --- /dev/null +++ b/dbms/src/Common/ThreadStatus.cpp @@ -0,0 +1,223 @@ +#include "ThreadStatus.h" +#include +#include +#include + +#include +#include + +#include + + +namespace ProfileEvents +{ + extern const Event RealTimeMicroseconds; + extern const Event RusageUserTimeMicroseconds; + extern const Event RusageSystemTimeMicroseconds; + extern const Event RusagePageReclaims; + extern const Event RusagePageVoluntaryContextSwitches; + extern const Event RusagePageInvoluntaryContextSwitches; +} + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +static UInt64 getCurrentTimeMicroseconds(clockid_t clock_type = CLOCK_MONOTONIC) +{ + struct timespec ts; + clock_gettime(clock_type, &ts); + return ts.tv_sec * 1000000ULL + ts.tv_nsec / 1000UL; +} + + +struct RusageCounters +{ + /// In microseconds + UInt64 real_time = 0; + UInt64 user_time = 0; + UInt64 sys_time = 0; + + UInt64 page_reclaims = 0; + UInt64 voluntary_context_switches = 0; + UInt64 involuntary_context_switches = 0; + + RusageCounters() = default; + RusageCounters(const struct rusage & rusage, UInt64 real_time_) + { + set(rusage, real_time_); + } + + static RusageCounters zeros(UInt64 real_time) + { + RusageCounters res; + res.real_time = real_time; + return res; + } + + static RusageCounters current() + { + RusageCounters res; + res.setFromCurrent(); + return res; + } + + void setFromCurrent() + { + struct rusage rusage; + getrusage(RUSAGE_THREAD, &rusage); + set(rusage, getCurrentTimeMicroseconds()); + } + + void set(const struct rusage & rusage, UInt64 real_time_) + { + real_time = real_time_; + user_time = rusage.ru_utime.tv_sec * 1000000UL + rusage.ru_utime.tv_usec; + sys_time = rusage.ru_utime.tv_sec * 1000000UL + rusage.ru_utime.tv_usec; + + page_reclaims = static_cast(rusage.ru_minflt); + voluntary_context_switches = static_cast(rusage.ru_nvcsw); + involuntary_context_switches = static_cast(rusage.ru_nivcsw); + } + + static void incrementProfileEvents(const RusageCounters & cur, const RusageCounters & prev) + { + ProfileEvents::increment(ProfileEvents::RealTimeMicroseconds, cur.real_time - prev.real_time); + ProfileEvents::increment(ProfileEvents::RusageUserTimeMicroseconds, cur.user_time - prev.user_time); + ProfileEvents::increment(ProfileEvents::RusageSystemTimeMicroseconds, cur.sys_time - prev.sys_time); + ProfileEvents::increment(ProfileEvents::RusagePageReclaims, cur.page_reclaims - prev.page_reclaims); + ProfileEvents::increment(ProfileEvents::RusagePageVoluntaryContextSwitches, cur.voluntary_context_switches - prev.voluntary_context_switches); + ProfileEvents::increment(ProfileEvents::RusagePageInvoluntaryContextSwitches, cur.involuntary_context_switches - prev.involuntary_context_switches); + } + + static void updateProfileEvents(RusageCounters & last_counters) + { + RusageCounters current = RusageCounters::current(); + RusageCounters::incrementProfileEvents(current, last_counters); + last_counters = current; + } +}; + + +struct ThreadStatus::Payload +{ + RusageCounters last_rusage; +}; + + +//static void QueryThreadStatusOnThreadFinish(void * arg) +//{ +// auto thread_status = static_cast(arg); +// thread_status->is_finished = true; +// LOG_DEBUG(thread_status->log, "Thread " << thread_status->poco_thread_number << " is finished"); +//} + + +ThreadStatus::ThreadStatus() + : poco_thread_number(Poco::ThreadNumber::get()), + performance_counters(ProfileEvents::Level::Thread), + log(&Poco::Logger::get("ThreadStatus")) +{ + LOG_DEBUG(log, "Thread " << poco_thread_number << " created"); +} + +ThreadStatus::~ThreadStatus() +{ + LOG_DEBUG(log, "Thread " << poco_thread_number << " destroyed"); +} + +void ThreadStatus::init(QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker) +{ + if (!initialized) + { + if (auto counters_parent = performance_counters.parent) + if (counters_parent != parent_counters) + LOG_WARNING(current_thread->log, "Parent performance counters are already set, overwrite"); + + if (auto tracker_parent = memory_tracker.getParent()) + if (tracker_parent != parent_memory_tracker) + LOG_WARNING(current_thread->log, "Parent memory tracker is already set, overwrite"); + + return; + } + + initialized = true; + parent_query = parent_query_; + performance_counters.parent = parent_counters; + memory_tracker.setParent(parent_memory_tracker); + memory_tracker.setDescription("(for thread)"); + + onStart(); +} + +void ThreadStatus::onStart() +{ + payload = std::make_shared(); + + /// First init of thread rusage counters, set real time to zero, other metrics remain as is + payload->last_rusage.setFromCurrent(); + RusageCounters::incrementProfileEvents(payload->last_rusage, RusageCounters::zeros(payload->last_rusage.real_time)); +} + +void ThreadStatus::onExit() +{ + if (!initialized || !payload) + return; + + RusageCounters::updateProfileEvents(payload->last_rusage); +} + +void ThreadStatus::reset() +{ + parent_query = nullptr; + performance_counters.reset(); + memory_tracker.reset(); + memory_tracker.setParent(nullptr); + initialized = false; +} + + +void ThreadStatus::setCurrentThreadParentQuery(QueryStatus * parent_process) +{ + if (!current_thread) + throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR); + + if (!parent_process) + { + current_thread->init(parent_process, nullptr, nullptr); + return; + } + + current_thread->init(parent_process, &parent_process->performance_counters, &parent_process->memory_tracker); + + { + std::lock_guard lock(parent_process->threads_mutex); + auto res = parent_process->thread_statuses.emplace(current_thread->poco_thread_number, current_thread); + + if (!res.second && res.first->second.get() != current_thread.get()) + throw Exception("Thread " + std::to_string(current_thread->poco_thread_number) + " is set twice", ErrorCodes::LOGICAL_ERROR); + } +} + +void ThreadStatus::setCurrentThreadFromSibling(const ThreadStatusPtr & sibling_thread) +{ + if (!current_thread) + throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR); + + if (sibling_thread == nullptr) + throw Exception("Sibling thread was not initialized", ErrorCodes::LOGICAL_ERROR); + + current_thread->init(sibling_thread->parent_query, sibling_thread->performance_counters.parent, sibling_thread->memory_tracker.getParent()); +} + + +thread_local ThreadStatusPtr current_thread = ThreadStatus::create(); + + +} diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h new file mode 100644 index 00000000000..bce75b70779 --- /dev/null +++ b/dbms/src/Common/ThreadStatus.h @@ -0,0 +1,54 @@ +#pragma once +#include +#include +#include +#include + +namespace Poco +{ + class Logger; +} + + +namespace DB +{ + +struct QueryStatus; +struct ThreadStatus; +using ThreadStatusPtr = std::shared_ptr; + + +struct ThreadStatus : public ext::shared_ptr_helper, public std::enable_shared_from_this +{ + UInt32 poco_thread_number = 0; + QueryStatus * parent_query = nullptr; + + ProfileEvents::Counters performance_counters; + MemoryTracker memory_tracker; + + void init(QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker); + void onStart(); + void onExit(); + + /// Reset all references and metrics + void reset(); + + static void setCurrentThreadParentQuery(QueryStatus * parent_process); + static void setCurrentThreadFromSibling(const ThreadStatusPtr & sibling_thread); + + ~ThreadStatus(); + +protected: + ThreadStatus(); + + bool initialized = false; + Poco::Logger * log; + + struct Payload; + std::shared_ptr payload; +}; + + +extern thread_local ThreadStatusPtr current_thread; + +} diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index 0a80628cf2a..d74bea070f1 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -7,6 +7,8 @@ #include #include #include +#include +#include namespace CurrentMetrics @@ -97,7 +99,8 @@ protected: /// If there were no calculations yet, calculate the first block synchronously if (!started) { - calculate(current_memory_tracker); + ThreadStatusPtr main_thread = current_thread; + calculate(main_thread); started = true; } else /// If the calculations are already in progress - wait for the result @@ -121,12 +124,12 @@ protected: void next() { ready.reset(); - pool.schedule(std::bind(&AsynchronousBlockInputStream::calculate, this, current_memory_tracker)); + pool.schedule([this, main_thread=current_thread] () { calculate(main_thread); }); } /// Calculations that can be performed in a separate thread - void calculate(MemoryTracker * memory_tracker) + void calculate(ThreadStatusPtr main_thread) { CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; @@ -135,8 +138,9 @@ protected: if (first) { first = false; + if (main_thread) + ThreadStatus::setCurrentThreadFromSibling(main_thread); setThreadName("AsyncBlockInput"); - current_memory_tracker = memory_tracker; children.back()->readPrefix(); } diff --git a/dbms/src/DataStreams/BlockIO.h b/dbms/src/DataStreams/BlockIO.h index 6d97e30e510..685fdef87c2 100644 --- a/dbms/src/DataStreams/BlockIO.h +++ b/dbms/src/DataStreams/BlockIO.h @@ -12,8 +12,7 @@ class ProcessListEntry; struct BlockIO { /** process_list_entry should be destroyed after in and after out, - * since in and out contain pointer to an object inside process_list_entry - * (MemoryTracker * current_memory_tracker), + * since in and out contain pointer to objects inside process_list_entry (query-level MemoryTracker for example), * which could be used before destroying of in and out. */ std::shared_ptr process_list_entry; diff --git a/dbms/src/DataStreams/CountingBlockOutputStream.h b/dbms/src/DataStreams/CountingBlockOutputStream.h index 0918d33f113..54fbeb67d2b 100644 --- a/dbms/src/DataStreams/CountingBlockOutputStream.h +++ b/dbms/src/DataStreams/CountingBlockOutputStream.h @@ -20,7 +20,7 @@ public: progress_callback = callback; } - void setProcessListElement(ProcessListElement * elem) + void setProcessListElement(QueryStatus * elem) { process_elem = elem; } @@ -43,7 +43,7 @@ protected: BlockOutputStreamPtr stream; Progress progress; ProgressCallback progress_callback; - ProcessListElement * process_elem = nullptr; + QueryStatus * process_elem = nullptr; }; } diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 4be77897486..561ca08c143 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -379,7 +379,7 @@ void IProfilingBlockInputStream::setProgressCallback(const ProgressCallback & ca } -void IProfilingBlockInputStream::setProcessListElement(ProcessListElement * elem) +void IProfilingBlockInputStream::setProcessListElement(QueryStatus * elem) { process_list_elem = elem; diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.h b/dbms/src/DataStreams/IProfilingBlockInputStream.h index 254b4e31018..1a989535377 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.h +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.h @@ -18,7 +18,7 @@ namespace ErrorCodes } class QuotaForIntervals; -struct ProcessListElement; +struct QueryStatus; class IProfilingBlockInputStream; using ProfilingBlockInputStreamPtr = std::shared_ptr; @@ -101,7 +101,7 @@ public: * Based on this information, the quota and some restrictions will be checked. * This information will also be available in the SHOW PROCESSLIST request. */ - void setProcessListElement(ProcessListElement * elem); + void setProcessListElement(QueryStatus * elem); /** Set the approximate total number of rows to read. */ @@ -192,7 +192,7 @@ protected: std::atomic is_cancelled{false}; bool is_killed{false}; ProgressCallback progress_callback; - ProcessListElement * process_list_elem = nullptr; + QueryStatus * process_list_elem = nullptr; /// Additional information that can be generated during the work process. diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index bf436bb8547..b65b5eb7e36 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -175,10 +176,11 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start() { auto & child = children[i]; - auto memory_tracker = current_memory_tracker; - reading_pool->schedule([&child, memory_tracker] + auto main_thread = current_thread; + reading_pool->schedule([&child, main_thread] { - current_memory_tracker = memory_tracker; + if (main_thread) + ThreadStatus::setCurrentThreadFromSibling(main_thread); setThreadName("MergeAggReadThr"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; child->readPrefix(); @@ -196,8 +198,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start() */ for (size_t i = 0; i < merging_threads; ++i) - pool.schedule(std::bind(&MergingAggregatedMemoryEfficientBlockInputStream::mergeThread, - this, current_memory_tracker)); + pool.schedule([this] () { mergeThread(current_thread); } ); } } @@ -293,10 +294,11 @@ void MergingAggregatedMemoryEfficientBlockInputStream::finalize() } -void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(MemoryTracker * memory_tracker) +void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadStatusPtr main_thread) { + if (main_thread) + ThreadStatus::setCurrentThreadFromSibling(main_thread); setThreadName("MergeAggMergThr"); - current_memory_tracker = memory_tracker; CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; try @@ -480,10 +482,11 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate { if (need_that_input(input)) { - auto memory_tracker = current_memory_tracker; - reading_pool->schedule([&input, &read_from_input, memory_tracker] + auto main_thread = current_thread; + reading_pool->schedule([&input, &read_from_input, main_thread] { - current_memory_tracker = memory_tracker; + if (main_thread) + ThreadStatus::setCurrentThreadFromSibling(main_thread); setThreadName("MergeAggReadThr"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; read_from_input(input); diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index 837c10869cf..5599f8e1120 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -151,7 +152,7 @@ private: std::unique_ptr parallel_merge_data; - void mergeThread(MemoryTracker * memory_tracker); + void mergeThread(ThreadStatusPtr main_thread); void finalize(); }; diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index 115dc6e1a3a..fdddf6b2149 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -12,6 +12,7 @@ #include #include #include +#include /** Allows to process multiple block input streams (sources) in parallel, using specified number of threads. @@ -103,10 +104,11 @@ public: /// Start background threads, start work. void process() { + QueryStatus * current_query = current_thread ? current_thread->parent_query : nullptr; active_threads = max_threads; threads.reserve(max_threads); for (size_t i = 0; i < max_threads; ++i) - threads.emplace_back(std::bind(&ParallelInputsProcessor::thread, this, current_memory_tracker, i)); + threads.emplace_back([=] () { thread(current_query, i); } ); } /// Ask all sources to stop earlier than they run out. @@ -174,9 +176,9 @@ private: } } - void thread(MemoryTracker * memory_tracker, size_t thread_num) + void thread(QueryStatus * query_status, size_t thread_num) { - current_memory_tracker = memory_tracker; + ThreadStatus::setCurrentThreadParentQuery(query_status); std::exception_ptr exception; setThreadName("ParalInputsProc"); diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index eebc95cada9..48d7cab01af 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -24,6 +24,7 @@ #include #include #include +#include "Common/ThreadStatus.h" namespace ProfileEvents @@ -139,8 +140,10 @@ Aggregator::Aggregator(const Params & params_) : params(params_), isCancelled([]() { return false; }) { - if (current_memory_tracker) - memory_usage_before_aggregation = current_memory_tracker->get(); + /// Use query-level memory tracker + if (current_thread) + if (auto memory_tracker = current_thread->memory_tracker.getParent()) + memory_usage_before_aggregation = memory_tracker->get(); aggregate_functions.resize(params.aggregates_size); for (size_t i = 0; i < params.aggregates_size; ++i) @@ -798,8 +801,9 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re size_t result_size = result.sizeWithoutOverflowRow(); Int64 current_memory_usage = 0; - if (current_memory_tracker) - current_memory_usage = current_memory_tracker->get(); + if (current_thread) + if (auto memory_tracker = current_thread->memory_tracker.getParent()) + current_memory_usage = memory_tracker->get(); auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation; /// Here all the results in the sum are taken into account, from different threads. @@ -1271,9 +1275,10 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( bool final, ThreadPool * thread_pool) const { - auto converter = [&](size_t bucket, MemoryTracker * memory_tracker) + auto converter = [&](size_t bucket, const ThreadStatusPtr & main_thread) { - current_memory_tracker = memory_tracker; + if (main_thread) + ThreadStatus::setCurrentThreadFromSibling(main_thread); return convertOneBucketToBlock(data_variants, method, final, bucket); }; @@ -1288,7 +1293,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( if (method.data.impls[bucket].empty()) continue; - tasks[bucket] = std::packaged_task(std::bind(converter, bucket, current_memory_tracker)); + tasks[bucket] = std::packaged_task(std::bind(converter, bucket, current_thread)); if (thread_pool) thread_pool->schedule([bucket, &tasks] { tasks[bucket](); }); @@ -1550,7 +1555,7 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl( template -void NO_INLINE Aggregator::mergeBucketImpl( +void NO_INLINE Aggregator:: mergeBucketImpl( ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena) const { /// We connect all aggregation results to the first. @@ -1717,13 +1722,14 @@ private: if (max_scheduled_bucket_num >= NUM_BUCKETS) return; - parallel_merge_data->pool.schedule(std::bind(&MergingAndConvertingBlockInputStream::thread, this, - max_scheduled_bucket_num, current_memory_tracker)); + parallel_merge_data->pool.schedule([this, main_thread=current_thread] () { thread(max_scheduled_bucket_num, main_thread); }); } - void thread(Int32 bucket_num, MemoryTracker * memory_tracker) + void thread(Int32 bucket_num, const ThreadStatusPtr & main_thread) { - current_memory_tracker = memory_tracker; + if (main_thread) + ThreadStatus::setCurrentThreadFromSibling(main_thread); + setThreadName("MergingAggregtd"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; @@ -2028,9 +2034,10 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV LOG_TRACE(log, "Merging partially aggregated two-level data."); - auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, MemoryTracker * memory_tracker) + auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, const ThreadStatusPtr & main_thread) { - current_memory_tracker = memory_tracker; + if (main_thread) + ThreadStatus::setCurrentThreadFromSibling(main_thread); for (Block & block : bucket_to_blocks[bucket]) { @@ -2064,7 +2071,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV result.aggregates_pools.push_back(std::make_shared()); Arena * aggregates_pool = result.aggregates_pools.back().get(); - auto task = std::bind(merge_bucket, bucket, aggregates_pool, current_memory_tracker); + auto task = std::bind(merge_bucket, bucket, aggregates_pool, current_thread); if (thread_pool) thread_pool->schedule(task); diff --git a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp index 609b3b170e2..abd8a1c15e3 100644 --- a/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/dbms/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -42,7 +42,7 @@ BlockInputStreams executeQuery( ThrottlerPtr user_level_throttler; if (settings.limits.max_network_bandwidth_for_user) if (auto process_list_element = context.getProcessListElement()) - if (auto user_process_list = process_list_element->user_process_list) + if (auto user_process_list = process_list_element->getUserProcessList()) user_level_throttler = user_process_list->user_throttler; /// Network bandwidth limit, if needed. diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 9204cdd23aa..440f1d54b56 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -45,7 +45,7 @@ class Compiler; class MarkCache; class UncompressedCache; class ProcessList; -struct ProcessListElement; +struct QueryStatus; class Macros; struct Progress; class Clusters; @@ -99,7 +99,7 @@ private: Settings settings; /// Setting for query execution. using ProgressCallback = std::function; ProgressCallback progress_callback; /// Callback for tracking progress of query execution. - ProcessListElement * process_list_elem = nullptr; /// For tracking total resource usage for query. + QueryStatus * process_list_elem = nullptr; /// For tracking total resource usage for query. String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification. /// Thus, used in HTTP interface. If not specified - then some globally default format is used. @@ -285,9 +285,9 @@ public: /** Set in executeQuery and InterpreterSelectQuery. Then it is used in IProfilingBlockInputStream, * to update and monitor information about the total number of resources spent for the query. */ - void setProcessListElement(ProcessListElement * elem); + void setProcessListElement(QueryStatus * elem); /// Can return nullptr if the query was not inserted into the ProcessList. - ProcessListElement * getProcessListElement() const; + QueryStatus * getProcessListElement() const; /// List all queries. ProcessList & getProcessList(); diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 527c3d495ea..2b7146d138f 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -66,7 +66,7 @@ void ExternalLoader::init(bool throw_on_error) { /// During synchronous loading of external dictionaries at moment of query execution, /// we should not use per query memory limit. - TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker; + auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerBlocker(); reloadAndUpdate(throw_on_error); } diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 95e279b6e12..faf90489aeb 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include namespace DB @@ -26,7 +28,7 @@ ProcessList::EntryPtr ProcessList::insert( { std::lock_guard lock(mutex); - if (!is_kill_query && max_size && cur_size >= max_size + if (!is_kill_query && max_size && processes.size() >= max_size && (!settings.queue_max_wait_ms.totalMilliseconds() || !have_space.tryWait(mutex, settings.queue_max_wait_ms.totalMilliseconds()))) throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MUCH_SIMULTANEOUS_QUERIES); @@ -70,36 +72,44 @@ ProcessList::EntryPtr ProcessList::insert( } } - ++cur_size; - - res = std::make_shared(*this, cont.emplace(cont.end(), + auto process_it = processes.emplace(processes.end(), query_, client_info, settings.limits.max_memory_usage, settings.memory_tracker_fault_probability, - priorities.insert(settings.priority))); + priorities.insert(settings.priority)); + + res = std::make_shared(*this, process_it); if (!client_info.current_query_id.empty()) { ProcessListForUser & user_process_list = user_to_queries[client_info.current_user]; - user_process_list.queries[client_info.current_query_id] = &res->get(); + user_process_list.queries[client_info.current_query_id] = &*process_it; - if (current_memory_tracker) + /// Limits are only raised (to be more relaxed) or set to something instead of zero, + /// because settings for different queries will interfere each other: + /// setting from one query effectively sets values for all other queries. + + /// Track memory usage for all simultaneously running queries. + /// You should specify this value in configuration for default profile, + /// not for specific users, sessions or queries, + /// because this setting is effectively global. + total_memory_tracker.setOrRaiseLimit(settings.limits.max_memory_usage_for_all_queries); + total_memory_tracker.setDescription("(total)"); + + /// Track memory usage for all simultaneously running queries from single user. + user_process_list.user_memory_tracker.setParent(&total_memory_tracker); + user_process_list.user_memory_tracker.setOrRaiseLimit(settings.limits.max_memory_usage_for_user); + user_process_list.user_memory_tracker.setDescription("(for user)"); + + /// Query-level memory tracker is already set in the QueryStatus constructor + + if (!current_thread) + throw Exception("Thread is not initialized", ErrorCodes::LOGICAL_ERROR); + + if (current_thread) { - /// Limits are only raised (to be more relaxed) or set to something instead of zero, - /// because settings for different queries will interfere each other: - /// setting from one query effectively sets values for all other queries. - - /// Track memory usage for all simultaneously running queries from single user. - user_process_list.user_memory_tracker.setOrRaiseLimit(settings.limits.max_memory_usage_for_user); - user_process_list.user_memory_tracker.setDescription("(for user)"); - current_memory_tracker->setNext(&user_process_list.user_memory_tracker); - - /// Track memory usage for all simultaneously running queries. - /// You should specify this value in configuration for default profile, - /// not for specific users, sessions or queries, - /// because this setting is effectively global. - total_memory_tracker.setOrRaiseLimit(settings.limits.max_memory_usage_for_all_queries); - total_memory_tracker.setDescription("(total)"); - user_process_list.user_memory_tracker.setNext(&total_memory_tracker); + current_thread->setCurrentThreadParentQuery(&*process_it); + current_thread->memory_tracker.setOrRaiseLimit(settings.limits.max_memory_usage); + current_thread->memory_tracker.setDescription("(for thread)"); } if (settings.limits.max_network_bandwidth_for_user && !user_process_list.user_throttler) @@ -108,7 +118,7 @@ ProcessList::EntryPtr ProcessList::insert( "Network bandwidth limit for a user exceeded."); } - res->get().user_process_list = &user_process_list; + process_it->setUserProcessList(&user_process_list); } } @@ -121,6 +131,26 @@ ProcessListEntry::~ProcessListEntry() /// Destroy all streams to avoid long lock of ProcessList it->releaseQueryStreams(); + /// Finalize all threads statuses + { + std::lock_guard lock(it->threads_mutex); + + for (auto & elem : it->thread_statuses) + { + auto & thread_status = elem.second; + thread_status->onExit(); + thread_status->reset(); + thread_status.reset(); + } + + it->thread_statuses.clear(); + } + + /// Also reset query master thread status + /// NOTE: we can't destroy it, since master threads are selected from fixed thread pool + if (current_thread) + current_thread->reset(); + std::lock_guard lock(parent.mutex); /// The order of removing memory_trackers is important. @@ -130,7 +160,7 @@ ProcessListEntry::~ProcessListEntry() bool is_cancelled = it->is_cancelled; /// This removes the memory_tracker of one request. - parent.cont.erase(it); + parent.processes.erase(it); auto user_process_list = parent.user_to_queries.find(user); if (user_process_list != parent.user_to_queries.end()) @@ -154,11 +184,10 @@ ProcessListEntry::~ProcessListEntry() parent.user_to_queries.erase(user_process_list); } - --parent.cur_size; parent.have_space.signal(); /// This removes memory_tracker for all requests. At this time, no other memory_trackers live. - if (parent.cur_size == 0) + if (parent.processes.size() == 0) { /// Reset MemoryTracker, similarly (see above). parent.total_memory_tracker.logPeakMemoryUsage(); @@ -167,7 +196,28 @@ ProcessListEntry::~ProcessListEntry() } -void ProcessListElement::setQueryStreams(const BlockIO & io) +QueryStatus::QueryStatus( + const String & query_, + const ClientInfo & client_info_, + size_t max_memory_usage, + double memory_tracker_fault_probability, + QueryPriorities::Handle && priority_handle_) + : + query(query_), + client_info(client_info_), + priority_handle(std::move(priority_handle_)), + performance_counters(ProfileEvents::Level::Process), + num_queries_increment{CurrentMetrics::Query} +{ + memory_tracker.setOrRaiseLimit(max_memory_usage); + memory_tracker.setDescription("(for query)"); + + if (memory_tracker_fault_probability) + memory_tracker.setFaultProbability(memory_tracker_fault_probability); +} + + +void QueryStatus::setQueryStreams(const BlockIO & io) { std::lock_guard lock(query_streams_mutex); @@ -176,7 +226,7 @@ void ProcessListElement::setQueryStreams(const BlockIO & io) query_streams_initialized = true; } -void ProcessListElement::releaseQueryStreams() +void QueryStatus::releaseQueryStreams() { std::lock_guard lock(query_streams_mutex); @@ -186,14 +236,14 @@ void ProcessListElement::releaseQueryStreams() query_stream_out.reset(); } -bool ProcessListElement::streamsAreReleased() +bool QueryStatus::streamsAreReleased() { std::lock_guard lock(query_streams_mutex); return query_streams_released; } -bool ProcessListElement::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const +bool QueryStatus::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const { std::lock_guard lock(query_streams_mutex); @@ -206,7 +256,15 @@ bool ProcessListElement::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutpu } -void ProcessList::addTemporaryTable(ProcessListElement & elem, const String & table_name, const StoragePtr & storage) +void QueryStatus::setUserProcessList(ProcessListForUser * user_process_list_) +{ + user_process_list = user_process_list_; + performance_counters.parent = &user_process_list->user_performance_counters; + memory_tracker.setParent(&user_process_list->user_memory_tracker); +} + + +void ProcessList::addTemporaryTable(QueryStatus & elem, const String & table_name, const StoragePtr & storage) { std::lock_guard lock(mutex); @@ -214,7 +272,7 @@ void ProcessList::addTemporaryTable(ProcessListElement & elem, const String & ta } -ProcessListElement * ProcessList::tryGetProcessListElement(const String & current_query_id, const String & current_user) +QueryStatus * ProcessList::tryGetProcessListElement(const String & current_query_id, const String & current_user) { auto user_it = user_to_queries.find(current_user); if (user_it != user_to_queries.end()) @@ -234,7 +292,7 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr { std::lock_guard lock(mutex); - ProcessListElement * elem = tryGetProcessListElement(current_query_id, current_user); + QueryStatus * elem = tryGetProcessListElement(current_query_id, current_user); if (!elem) return CancellationCode::NotFound; @@ -260,4 +318,10 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr return CancellationCode::QueryIsNotInitializedYet; } + +ProcessListForUser::ProcessListForUser() +: user_performance_counters(ProfileEvents::Level::User, &ProfileEvents::global_counters) +{} + + } diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index f061e43ba15..c598c9c2b5d 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -5,16 +5,21 @@ #include #include #include +#include + #include -#include #include #include +#include #include +#include +#include +#include #include #include -#include +#include #include -#include +#include "ThreadPerformanceProfile.h" namespace CurrentMetrics @@ -30,7 +35,10 @@ using StoragePtr = std::shared_ptr; using Tables = std::map; struct Settings; class IAST; + struct ProcessListForUser; +struct QueryStatus; +struct ThreadStatus; /** List of currently executing queries. @@ -40,7 +48,7 @@ struct ProcessListForUser; /** Information of process list element. * To output in SHOW PROCESSLIST query. Does not contain any complex objects, that do something on copy or destructor. */ -struct ProcessInfo +struct QueryStatusInfo { String query; double elapsed_seconds; @@ -55,7 +63,7 @@ struct ProcessInfo /// Query and information about its execution. -struct ProcessListElement +struct QueryStatus { String query; ClientInfo client_info; @@ -67,19 +75,25 @@ struct ProcessListElement /// Progress of output stream Progress progress_out; - MemoryTracker memory_tracker; - QueryPriorities::Handle priority_handle; - CurrentMetrics::Increment num_queries {CurrentMetrics::Query}; + ProfileEvents::Counters performance_counters; + MemoryTracker memory_tracker; + + mutable std::shared_mutex threads_mutex; + using QueryThreadStatuses = std::map; /// Key is Poco's thread_id + QueryThreadStatuses thread_statuses; + + CurrentMetrics::Increment num_queries_increment{CurrentMetrics::Query}; bool is_cancelled = false; /// Temporary tables could be registered here. Modify under mutex. Tables temporary_tables; - /// Be careful using it. For example, queries field could be modified concurrently. - const ProcessListForUser * user_process_list = nullptr; + void setUserProcessList(ProcessListForUser * user_process_list_); + /// Be careful using it. For example, queries field of ProcessListForUser could be modified concurrently. + const ProcessListForUser * getUserProcessList() const { return user_process_list; } protected: @@ -94,27 +108,20 @@ protected: bool query_streams_initialized{false}; bool query_streams_released{false}; + ProcessListForUser * user_process_list = nullptr; + public: - ProcessListElement( + QueryStatus( const String & query_, const ClientInfo & client_info_, size_t max_memory_usage, double memory_tracker_fault_probability, - QueryPriorities::Handle && priority_handle_) - : query(query_), client_info(client_info_), memory_tracker(max_memory_usage), - priority_handle(std::move(priority_handle_)) - { - memory_tracker.setDescription("(for query)"); - current_memory_tracker = &memory_tracker; + QueryPriorities::Handle && priority_handle_); - if (memory_tracker_fault_probability) - memory_tracker.setFaultProbability(memory_tracker_fault_probability); - } - - ~ProcessListElement() + ~QueryStatus() { - current_memory_tracker = nullptr; + // TODO: master thread should be reset } bool updateProgressIn(const Progress & value) @@ -134,9 +141,9 @@ public: } - ProcessInfo getInfo() const + QueryStatusInfo getInfo() const { - ProcessInfo res; + QueryStatusInfo res; res.query = query; res.client_info = client_info; @@ -168,10 +175,13 @@ public: /// Data about queries for one user. struct ProcessListForUser { + ProcessListForUser(); + /// Query_id -> ProcessListElement * - using QueryToElement = std::unordered_map; + using QueryToElement = std::unordered_map; QueryToElement queries; + ProfileEvents::Counters user_performance_counters; /// Limit and counter for memory of all simultaneously running queries of single user. MemoryTracker user_memory_tracker; @@ -187,7 +197,7 @@ class ProcessList; class ProcessListEntry { private: - using Container = std::list; + using Container = std::list; ProcessList & parent; Container::iterator it; @@ -197,11 +207,11 @@ public: ~ProcessListEntry(); - ProcessListElement * operator->() { return &*it; } - const ProcessListElement * operator->() const { return &*it; } + QueryStatus * operator->() { return &*it; } + const QueryStatus * operator->() const { return &*it; } - ProcessListElement & get() { return *it; } - const ProcessListElement & get() const { return *it; } + QueryStatus & get() { return *it; } + const QueryStatus & get() const { return *it; } }; @@ -209,12 +219,12 @@ class ProcessList { friend class ProcessListEntry; public: - using Element = ProcessListElement; + using Element = QueryStatus; using Entry = ProcessListEntry; /// list, for iterators not to invalidate. NOTE: could replace with cyclic buffer, but not worth. using Container = std::list; - using Info = std::vector; + using Info = std::vector; /// User -> queries using UserToQueries = std::unordered_map; @@ -223,8 +233,7 @@ private: mutable Poco::Condition have_space; /// Number of currently running queries has become less than maximum. /// List of queries - Container cont; - size_t cur_size; /// In C++03 or C++11 and old ABI, std::list::size is not O(1). + Container processes; size_t max_size; /// 0 means no limit. Otherwise, when limit exceeded, an exception is thrown. /// Stores per-user info: queries, statistics and limits @@ -237,10 +246,10 @@ private: MemoryTracker total_memory_tracker; /// Call under lock. Finds process with specified current_user and current_query_id. - ProcessListElement * tryGetProcessListElement(const String & current_query_id, const String & current_user); + QueryStatus * tryGetProcessListElement(const String & current_query_id, const String & current_user); public: - ProcessList(size_t max_size_ = 0) : cur_size(0), max_size(max_size_) {} + ProcessList(size_t max_size_ = 0) : max_size(max_size_) {} using EntryPtr = std::shared_ptr; @@ -252,7 +261,7 @@ public: EntryPtr insert(const String & query_, const IAST * ast, const ClientInfo & client_info, const Settings & settings); /// Number of currently executing queries. - size_t size() const { return cur_size; } + size_t size() const { return processes.size(); } /// Get current state of process list. Info getInfo() const @@ -260,8 +269,8 @@ public: std::lock_guard lock(mutex); Info res; - res.reserve(cur_size); - for (const auto & elem : cont) + res.reserve(processes.size()); + for (const auto & elem : processes) res.emplace_back(elem.getInfo()); return res; @@ -274,7 +283,7 @@ public: } /// Register temporary table. Then it is accessible by query_id and name. - void addTemporaryTable(ProcessListElement & elem, const String & table_name, const StoragePtr & storage); + void addTemporaryTable(QueryStatus & elem, const String & table_name, const StoragePtr & storage); enum class CancellationCode { diff --git a/dbms/src/Interpreters/ThreadPerformanceProfile.h b/dbms/src/Interpreters/ThreadPerformanceProfile.h new file mode 100644 index 00000000000..b467f7f21b5 --- /dev/null +++ b/dbms/src/Interpreters/ThreadPerformanceProfile.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include +#include + +#include +#include + +#include + +#include +#include +#include + + + +namespace DB +{ + + + +} diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index e0d6c8a3fd6..f9c7358ae57 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -272,7 +272,7 @@ static std::tuple executeQueryImpl( /// Also make possible for caller to log successful query finish and exception during execution. res.finish_callback = [elem, &context, log_queries] (IBlockInputStream * stream_in, IBlockOutputStream * stream_out) mutable { - ProcessListElement * process_list_elem = context.getProcessListElement(); + QueryStatus * process_list_elem = context.getProcessListElement(); if (!process_list_elem) return; @@ -337,7 +337,7 @@ static std::tuple executeQueryImpl( elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time); elem.exception = getCurrentExceptionMessage(false); - ProcessListElement * process_list_elem = context.getProcessListElement(); + QueryStatus * process_list_elem = context.getProcessListElement(); if (process_list_elem) { diff --git a/dbms/src/Server/MetricsTransmitter.cpp b/dbms/src/Server/MetricsTransmitter.cpp index 278347a6774..ed87eb5cf9f 100644 --- a/dbms/src/Server/MetricsTransmitter.cpp +++ b/dbms/src/Server/MetricsTransmitter.cpp @@ -81,7 +81,7 @@ void MetricsTransmitter::transmit(std::vector & prev_count { for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { - const auto counter = ProfileEvents::counters[i].load(std::memory_order_relaxed); + const auto counter = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); const auto counter_increment = counter - prev_counters[i]; prev_counters[i] = counter; diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 1155d4c6362..5ce3afe3071 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -32,6 +32,8 @@ #include #include #include +#include + namespace CurrentMetrics @@ -184,8 +186,8 @@ void DistributedBlockOutputStream::waitForJobs() ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutputStream::JobInfo & job) { - auto memory_tracker = current_memory_tracker; - return [this, memory_tracker, &job]() + auto main_thread = current_thread; + return [this, main_thread, &job]() { SCOPE_EXIT({ std::lock_guard lock(mutex); @@ -193,9 +195,9 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp cond_var.notify_one(); }); - if (!current_memory_tracker) + if (current_thread) { - current_memory_tracker = memory_tracker; + ThreadStatus::setCurrentThreadFromSibling(main_thread); setThreadName("DistrOutStrProc"); } diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index 12b7edf32dc..1306f0d2d16 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -112,10 +113,8 @@ BackgroundProcessingPool::~BackgroundProcessingPool() void BackgroundProcessingPool::threadFunction() { setThreadName("BackgrProcPool"); - - MemoryTracker memory_tracker; - memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool); - current_memory_tracker = &memory_tracker; + if (current_thread) + ThreadStatus::setCurrentThreadParentQuery(nullptr); pcg64 rng(randomSeed()); std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, sleep_seconds_random_part)(rng))); @@ -199,8 +198,6 @@ void BackgroundProcessingPool::threadFunction() task->iterator = tasks.emplace(next_time_to_execute, task); } } - - current_memory_tracker = nullptr; } } diff --git a/dbms/src/Storages/MergeTree/MergeList.cpp b/dbms/src/Storages/MergeTree/MergeList.cpp index 9714d1b80da..48c3a72da29 100644 --- a/dbms/src/Storages/MergeTree/MergeList.cpp +++ b/dbms/src/Storages/MergeTree/MergeList.cpp @@ -1,6 +1,8 @@ #include #include #include +#include + namespace CurrentMetrics { @@ -20,11 +22,12 @@ MergeListElement::MergeListElement(const std::string & database, const std::stri source_part_names.emplace_back(source_part->name); /// Each merge is executed into separate background processing pool thread - background_pool_task_memory_tracker = current_memory_tracker; - if (background_pool_task_memory_tracker) + background_thread_memory_tracker = current_thread ? ¤t_thread->memory_tracker : nullptr; + if (background_thread_memory_tracker) { memory_tracker.setMetric(CurrentMetrics::MemoryTrackingForMerges); - background_pool_task_memory_tracker->setNext(&memory_tracker); + background_thread_memory_tracker_prev_parent = background_thread_memory_tracker->getParent(); + background_thread_memory_tracker->setParent(&memory_tracker); } } @@ -56,8 +59,8 @@ MergeInfo MergeListElement::getInfo() const MergeListElement::~MergeListElement() { /// Unplug memory_tracker from current background processing pool thread - if (background_pool_task_memory_tracker) - background_pool_task_memory_tracker->setNext(nullptr); + if (background_thread_memory_tracker) + background_thread_memory_tracker->setParent(background_thread_memory_tracker_prev_parent); } } diff --git a/dbms/src/Storages/MergeTree/MergeList.h b/dbms/src/Storages/MergeTree/MergeList.h index ed7241e4482..464cca89812 100644 --- a/dbms/src/Storages/MergeTree/MergeList.h +++ b/dbms/src/Storages/MergeTree/MergeList.h @@ -66,7 +66,8 @@ struct MergeListElement : boost::noncopyable std::atomic columns_written{}; MemoryTracker memory_tracker; - MemoryTracker * background_pool_task_memory_tracker; + MemoryTracker * background_thread_memory_tracker; + MemoryTracker * background_thread_memory_tracker_prev_parent = nullptr; /// Poco thread number used in logs UInt32 thread_number; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 63d763d5939..1d9a774a407 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -267,7 +267,7 @@ void MergeTreeReader::Stream::loadMarks() auto load = [&]() -> MarkCache::MappedPtr { /// Memory for marks must not be accounted as memory usage for query, because they are stored in shared cache. - TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker; + auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerBlocker(); size_t file_size = Poco::File(path).getSize(); size_t expected_file_size = sizeof(MarkInCompressedFile) * marks_count; diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index d3162810a1c..d4d7e2dea02 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -438,7 +438,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm * And otherwise it will look like excessively growing memory consumption in context of query. * (observed in long INSERT SELECTs) */ - TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker; + auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerBlocker(); /// Write index. The index contains Primary Key value for each `index_granularity` row. for (size_t i = index_offset; i < rows; i += storage.index_granularity) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index f2a4128e514..267588cc78b 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -180,7 +180,7 @@ static void appendBlock(const Block & from, Block & to) try { /// Avoid "memory limit exceeded" exceptions during rollback. - TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker; + auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerBlocker(); for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no) { diff --git a/dbms/src/Storages/System/StorageSystemEvents.cpp b/dbms/src/Storages/System/StorageSystemEvents.cpp index 96a02d72332..e712d1260a2 100644 --- a/dbms/src/Storages/System/StorageSystemEvents.cpp +++ b/dbms/src/Storages/System/StorageSystemEvents.cpp @@ -36,7 +36,7 @@ BlockInputStreams StorageSystemEvents::read( for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { - UInt64 value = ProfileEvents::counters[i]; + UInt64 value = ProfileEvents::global_counters[i]; if (0 != value) { From 364f519eea2247b1f26b78297def08e569fcce41 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Thu, 1 Mar 2018 19:52:24 +0300 Subject: [PATCH 002/315] Threads and ProfileEvents in system.processes. [#CLICKHOUSE-2910] --- .../AggregateFunctionCount.h | 3 - dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Common/MemoryTracker.cpp | 4 +- dbms/src/Common/MemoryTracker.h | 2 +- dbms/src/Common/ProfileEvents.cpp | 7 ++ dbms/src/Common/ProfileEvents.h | 4 +- dbms/src/Common/ThreadStatus.cpp | 102 +++++++++++++---- dbms/src/Common/ThreadStatus.h | 16 ++- dbms/src/Interpreters/Aggregator.cpp | 21 ++-- dbms/src/Interpreters/ProcessList.cpp | 13 +-- dbms/src/Interpreters/ProcessList.h | 38 +++++-- dbms/src/Interpreters/SystemLog.h | 2 +- .../System/StorageSystemProcesses.cpp | 59 +++++++++- .../Storages/System/StorageSystemProcesses.h | 3 +- .../Storages/System/StorageSystemTables.cpp | 103 +----------------- .../src/Storages/System/StorageSystemTables.h | 10 +- .../System/VirtualColumnsProcessor.cpp | 95 ++++++++++++++++ .../Storages/System/VirtualColumnsProcessor.h | 62 +++++++++++ ...5_storage_file_and_clickhouse-local_app.sh | 2 +- 19 files changed, 366 insertions(+), 181 deletions(-) create mode 100644 dbms/src/Storages/System/VirtualColumnsProcessor.cpp create mode 100644 dbms/src/Storages/System/VirtualColumnsProcessor.h diff --git a/dbms/src/AggregateFunctions/AggregateFunctionCount.h b/dbms/src/AggregateFunctions/AggregateFunctionCount.h index c903b03e675..5d39addff8b 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionCount.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionCount.h @@ -42,9 +42,6 @@ public: void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { - if (!place || !rhs) - throw Exception("nullptr", ErrorCodes::LOGICAL_ERROR); - data(place).count += data(rhs).count; } diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 4a253fc1895..9a2482530bb 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -370,6 +370,7 @@ namespace ErrorCodes extern const int QUERY_IS_PROHIBITED = 392; extern const int THERE_IS_NO_QUERY = 393; extern const int QUERY_WAS_CANCELLED = 394; + extern const int PTHREAD_ERROR = 395; extern const int KEEPER_EXCEPTION = 999; diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index c1a7d520acb..5731663b5c7 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -175,7 +175,7 @@ namespace CurrentMemoryTracker } } -DB::ActionBlockerSingleThread::BlockHolder getCurrentMemoryTrackerBlocker() +DB::ActionBlockerSingleThread::LockHolder getCurrentMemoryTrackerBlocker() { - return (DB::current_thread) ? DB::current_thread->memory_tracker.blocker.cancel() : DB::ActionBlockerSingleThread::BlockHolder(nullptr); + return (DB::current_thread) ? DB::current_thread->memory_tracker.blocker.cancel() : DB::ActionBlockerSingleThread::LockHolder(nullptr); } diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index 573e829b6a3..3e32ce5c9e5 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -123,4 +123,4 @@ namespace CurrentMemoryTracker } -DB::ActionBlockerSingleThread::BlockHolder getCurrentMemoryTrackerBlocker(); +DB::ActionBlockerSingleThread::LockHolder getCurrentMemoryTrackerBlocker(); diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 9c1882c38de..a2737a3925f 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -180,6 +180,13 @@ void Counters::reset() resetCounters(); } +void Counters::getPartiallyAtomicSnapshot(Counters & res) const +{ + for (Event i = 0; i < num_counters; ++i) + res.counters[i].store(counters[i], std::memory_order_relaxed); +} + + const char * getDescription(Event event) { static const char * descriptions[] = diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index 302e5be1884..174c2945435 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -29,7 +29,7 @@ namespace ProfileEvents { Counter * counters = nullptr; Counters * parent = nullptr; - const Level level = Level::Thread; + Level level = Level::Thread; std::unique_ptr counters_holder; Counters() = default; @@ -55,6 +55,8 @@ namespace ProfileEvents } while (current != nullptr); } + void getPartiallyAtomicSnapshot(Counters & res) const; + /// Reset metrics and parent void reset(); diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 52e0a29f3c9..b52099845b0 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -5,6 +5,7 @@ #include #include +#include #include @@ -26,6 +27,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int PTHREAD_ERROR; } @@ -105,7 +107,7 @@ struct RusageCounters }; -struct ThreadStatus::Payload +struct ThreadStatus::Impl { RusageCounters last_rusage; }; @@ -118,13 +120,40 @@ struct ThreadStatus::Payload // LOG_DEBUG(thread_status->log, "Thread " << thread_status->poco_thread_number << " is finished"); //} +static pthread_once_t once_query_at_exit_callback = PTHREAD_ONCE_INIT; +static pthread_key_t tid_key_at_exit; + +static void thread_destructor(void * data) +{ + auto thread_status = static_cast(data); + thread_status->onExit(); + LOG_DEBUG(thread_status->log, "Destruct thread " << thread_status->poco_thread_number); + thread_status->thread_exited = true; +} + +static void thread_create_at_exit_key() { + if (0 != pthread_key_create(&tid_key_at_exit, thread_destructor)) + throw Exception("Failed pthread_key_create", ErrorCodes::PTHREAD_ERROR); +} + ThreadStatus::ThreadStatus() : poco_thread_number(Poco::ThreadNumber::get()), performance_counters(ProfileEvents::Level::Thread), log(&Poco::Logger::get("ThreadStatus")) { + impl = std::make_shared(); + LOG_DEBUG(log, "Thread " << poco_thread_number << " created"); + + if (0 != pthread_once(&once_query_at_exit_callback, thread_create_at_exit_key)) + throw Exception("Failed pthread_once", ErrorCodes::PTHREAD_ERROR); + + if (nullptr != pthread_getspecific(tid_key_at_exit)) + throw Exception("pthread_getspecific is already set", ErrorCodes::LOGICAL_ERROR); + + if (0 != pthread_setspecific(tid_key_at_exit, static_cast(this))) + throw Exception("Failed pthread_setspecific", ErrorCodes::PTHREAD_ERROR); } ThreadStatus::~ThreadStatus() @@ -134,52 +163,59 @@ ThreadStatus::~ThreadStatus() void ThreadStatus::init(QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker) { - if (!initialized) + if (initialized) { if (auto counters_parent = performance_counters.parent) if (counters_parent != parent_counters) - LOG_WARNING(current_thread->log, "Parent performance counters are already set, overwrite"); + LOG_WARNING(log, "Parent performance counters are already set, overwrite"); if (auto tracker_parent = memory_tracker.getParent()) if (tracker_parent != parent_memory_tracker) - LOG_WARNING(current_thread->log, "Parent memory tracker is already set, overwrite"); + LOG_WARNING(log, "Parent memory tracker is already set, overwrite"); return; } - initialized = true; parent_query = parent_query_; performance_counters.parent = parent_counters; memory_tracker.setParent(parent_memory_tracker); memory_tracker.setDescription("(for thread)"); + initialized = true; + + /// Attach current thread to list of query threads + if (parent_query) + { + std::lock_guard lock(parent_query->threads_mutex); + auto res = parent_query->thread_statuses.emplace(current_thread->poco_thread_number, current_thread); + + if (!res.second && res.first->second.get() != current_thread.get()) + throw Exception("Thread " + std::to_string(current_thread->poco_thread_number) + " is set twice", ErrorCodes::LOGICAL_ERROR); + } onStart(); } void ThreadStatus::onStart() { - payload = std::make_shared(); - /// First init of thread rusage counters, set real time to zero, other metrics remain as is - payload->last_rusage.setFromCurrent(); - RusageCounters::incrementProfileEvents(payload->last_rusage, RusageCounters::zeros(payload->last_rusage.real_time)); + impl->last_rusage.setFromCurrent(); + RusageCounters::incrementProfileEvents(impl->last_rusage, RusageCounters::zeros(impl->last_rusage.real_time)); } void ThreadStatus::onExit() { - if (!initialized || !payload) - return; - - RusageCounters::updateProfileEvents(payload->last_rusage); + RusageCounters::updateProfileEvents(impl->last_rusage); } void ThreadStatus::reset() { + std::lock_guard lock(mutex); + + initialized = false; parent_query = nullptr; performance_counters.reset(); memory_tracker.reset(); memory_tracker.setParent(nullptr); - initialized = false; } @@ -190,19 +226,11 @@ void ThreadStatus::setCurrentThreadParentQuery(QueryStatus * parent_process) if (!parent_process) { - current_thread->init(parent_process, nullptr, nullptr); + current_thread->init(nullptr, nullptr, nullptr); return; } current_thread->init(parent_process, &parent_process->performance_counters, &parent_process->memory_tracker); - - { - std::lock_guard lock(parent_process->threads_mutex); - auto res = parent_process->thread_statuses.emplace(current_thread->poco_thread_number, current_thread); - - if (!res.second && res.first->second.get() != current_thread.get()) - throw Exception("Thread " + std::to_string(current_thread->poco_thread_number) + " is set twice", ErrorCodes::LOGICAL_ERROR); - } } void ThreadStatus::setCurrentThreadFromSibling(const ThreadStatusPtr & sibling_thread) @@ -213,11 +241,39 @@ void ThreadStatus::setCurrentThreadFromSibling(const ThreadStatusPtr & sibling_t if (sibling_thread == nullptr) throw Exception("Sibling thread was not initialized", ErrorCodes::LOGICAL_ERROR); + std::lock_guard lock(sibling_thread->mutex); current_thread->init(sibling_thread->parent_query, sibling_thread->performance_counters.parent, sibling_thread->memory_tracker.getParent()); } +struct ScopeCurrentThread +{ + ScopeCurrentThread() + { + if (!current_thread) + std::terminate(); // current_thread must be initialized + } + + ~ScopeCurrentThread() + { + if (!current_thread) + std::terminate(); // current_thread must be initialized + + if (Poco::ThreadNumber::get() != current_thread->poco_thread_number) + std::terminate(); // unexpected thread number + + current_thread->onExit(); + LOG_DEBUG(current_thread->log, "Thread " << current_thread->poco_thread_number << " is exiting"); + current_thread->thread_exited = true; + } +}; + thread_local ThreadStatusPtr current_thread = ThreadStatus::create(); +/// Order of current_thread and current_thread_scope matters +static thread_local ScopeCurrentThread current_thread_scope; + + + } diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index bce75b70779..ee0d2072b9c 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -3,6 +3,8 @@ #include #include #include +#include + namespace Poco { @@ -15,16 +17,18 @@ namespace DB struct QueryStatus; struct ThreadStatus; +struct ScopeCurrentThread; using ThreadStatusPtr = std::shared_ptr; -struct ThreadStatus : public ext::shared_ptr_helper, public std::enable_shared_from_this +struct ThreadStatus : public ext::shared_ptr_helper { UInt32 poco_thread_number = 0; QueryStatus * parent_query = nullptr; - ProfileEvents::Counters performance_counters; MemoryTracker memory_tracker; + bool thread_exited = false; + std::mutex mutex; void init(QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker); void onStart(); @@ -38,14 +42,16 @@ struct ThreadStatus : public ext::shared_ptr_helper, public std::e ~ThreadStatus(); -protected: + friend struct ScopeCurrentThread; + +//protected: ThreadStatus(); bool initialized = false; Poco::Logger * log; - struct Payload; - std::shared_ptr payload; + struct Impl; + std::shared_ptr impl; }; diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 48d7cab01af..a8e3f14cc18 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1275,10 +1275,9 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( bool final, ThreadPool * thread_pool) const { - auto converter = [&](size_t bucket, const ThreadStatusPtr & main_thread) + auto converter = [&](size_t bucket, ThreadStatusPtr main_thread) { - if (main_thread) - ThreadStatus::setCurrentThreadFromSibling(main_thread); + ThreadStatus::setCurrentThreadFromSibling(main_thread); return convertOneBucketToBlock(data_variants, method, final, bucket); }; @@ -1555,7 +1554,7 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl( template -void NO_INLINE Aggregator:: mergeBucketImpl( +void NO_INLINE Aggregator::mergeBucketImpl( ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena) const { /// We connect all aggregation results to the first. @@ -1722,14 +1721,13 @@ private: if (max_scheduled_bucket_num >= NUM_BUCKETS) return; - parallel_merge_data->pool.schedule([this, main_thread=current_thread] () { thread(max_scheduled_bucket_num, main_thread); }); + parallel_merge_data->pool.schedule(std::bind(&MergingAndConvertingBlockInputStream::thread, this, + max_scheduled_bucket_num, current_thread)); } - void thread(Int32 bucket_num, const ThreadStatusPtr & main_thread) + void thread(Int32 bucket_num, ThreadStatusPtr main_thread) { - if (main_thread) - ThreadStatus::setCurrentThreadFromSibling(main_thread); - + ThreadStatus::setCurrentThreadFromSibling(main_thread); setThreadName("MergingAggregtd"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; @@ -2034,10 +2032,9 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV LOG_TRACE(log, "Merging partially aggregated two-level data."); - auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, const ThreadStatusPtr & main_thread) + auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadStatusPtr main_thread) { - if (main_thread) - ThreadStatus::setCurrentThreadFromSibling(main_thread); + ThreadStatus::setCurrentThreadFromSibling(main_thread); for (Block & block : bucket_to_blocks[bucket]) { diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index faf90489aeb..9a71ceda434 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -133,24 +133,17 @@ ProcessListEntry::~ProcessListEntry() /// Finalize all threads statuses { + current_thread->onExit(); + std::lock_guard lock(it->threads_mutex); for (auto & elem : it->thread_statuses) { auto & thread_status = elem.second; - thread_status->onExit(); thread_status->reset(); - thread_status.reset(); - } - - it->thread_statuses.clear(); + }; } - /// Also reset query master thread status - /// NOTE: we can't destroy it, since master threads are selected from fixed thread pool - if (current_thread) - current_thread->reset(); - std::lock_guard lock(parent.mutex); /// The order of removing memory_trackers is important. diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index c598c9c2b5d..1b2f8c63552 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -59,6 +59,10 @@ struct QueryStatusInfo size_t written_bytes; Int64 memory_usage; ClientInfo client_info; + + /// Optional fields, filled by request + std::vector thread_numbers; + std::unique_ptr profile_counters; }; @@ -81,7 +85,8 @@ struct QueryStatus MemoryTracker memory_tracker; mutable std::shared_mutex threads_mutex; - using QueryThreadStatuses = std::map; /// Key is Poco's thread_id + /// Key is Poco's thread_id + using QueryThreadStatuses = std::map; QueryThreadStatuses thread_statuses; CurrentMetrics::Increment num_queries_increment{CurrentMetrics::Query}; @@ -264,16 +269,35 @@ public: size_t size() const { return processes.size(); } /// Get current state of process list. - Info getInfo() const + Info getInfo(bool get_thread_list = false, bool get_profile_events = false) const { + Info per_query_infos; + std::lock_guard lock(mutex); - Info res; - res.reserve(processes.size()); - for (const auto & elem : processes) - res.emplace_back(elem.getInfo()); + per_query_infos.reserve(processes.size()); + for (const auto & process : processes) + { + per_query_infos.emplace_back(process.getInfo()); + QueryStatusInfo & current_info = per_query_infos.back(); - return res; + if (get_thread_list) + { + std::lock_guard lock(process.threads_mutex); + current_info.thread_numbers.reserve(process.thread_statuses.size()); + + for (auto & thread_status_elem : process.thread_statuses) + current_info.thread_numbers.emplace_back(thread_status_elem.second->poco_thread_number); + } + + if (get_profile_events) + { + current_info.profile_counters = std::make_unique(ProfileEvents::Level::Process); + process.performance_counters.getPartiallyAtomicSnapshot(*current_info.profile_counters); + } + } + + return per_query_infos; } void setMaxSize(size_t max_size_) diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index faba63bc2ad..6d970a47f71 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -56,7 +56,7 @@ class QueryLog; class PartLog; -/// System logs should be destroyed in destructor of last Context and before tables, +/// System logs should be destroyed in destructor of the last Context and before tables, /// because SystemLog destruction makes insert query while flushing data into underlying tables struct SystemLogs { diff --git a/dbms/src/Storages/System/StorageSystemProcesses.cpp b/dbms/src/Storages/System/StorageSystemProcesses.cpp index af02417af4f..4596ec583cb 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.cpp +++ b/dbms/src/Storages/System/StorageSystemProcesses.cpp @@ -1,10 +1,13 @@ #include #include #include +#include +#include #include #include #include #include +#include namespace DB @@ -50,6 +53,18 @@ StorageSystemProcesses::StorageSystemProcesses(const std::string & name_) { "memory_usage", std::make_shared() }, { "query", std::make_shared() } }; + + virtual_columns = ColumnsWithTypeAndName{ + { + std::make_shared(std::make_shared()), + "thread_numbers" + }, + { + std::make_shared(std::make_shared( + DataTypes{std::make_shared(), std::make_shared()})), + "profile_counters" + } + }; } @@ -61,12 +76,18 @@ BlockInputStreams StorageSystemProcesses::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - check(column_names); processed_stage = QueryProcessingStage::FetchColumns; - ProcessList::Info info = context.getProcessList().getInfo(); + auto virtual_columns_processor = getVirtualColumnsProcessor(); + bool has_thread_numbers, has_profile_counters; + Names real_columns = virtual_columns_processor.process(column_names, {&has_thread_numbers, &has_profile_counters}); + check(real_columns); - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); + Block res_block = getSampleBlock().cloneEmpty(); + virtual_columns_processor.appendVirtualColumns(res_block); + MutableColumns res_columns = res_block.cloneEmptyColumns(); + + ProcessList::Info info = context.getProcessList().getInfo(has_thread_numbers, has_profile_counters); for (const auto & process : info) { @@ -98,10 +119,38 @@ BlockInputStreams StorageSystemProcesses::read( res_columns[i++]->insert(UInt64(process.written_bytes)); res_columns[i++]->insert(process.memory_usage); res_columns[i++]->insert(process.query); + + if (has_thread_numbers) + { + Array thread_numbers; + thread_numbers.reserve(process.thread_numbers.size()); + + for (const UInt32 thread_number : process.thread_numbers) + thread_numbers.emplace_back(UInt64(thread_number)); + + res_columns[i++]->insert(std::move(thread_numbers)); + } + + if (has_profile_counters) + { + Array profile_counters; + profile_counters.reserve(ProfileEvents::Counters::num_counters); + + for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) + { + Array name_and_counter{ + String(ProfileEvents::getDescription(event)), + UInt64((*process.profile_counters)[event].load(std::memory_order_relaxed)) + }; + + profile_counters.emplace_back(Tuple(std::move(name_and_counter))); + } + + res_columns[i++]->insert(std::move(profile_counters)); + } } - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); + return BlockInputStreams(1, std::make_shared(res_block.cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemProcesses.h b/dbms/src/Storages/System/StorageSystemProcesses.h index f8f26d13d35..59e7fa91d16 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.h +++ b/dbms/src/Storages/System/StorageSystemProcesses.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -12,7 +13,7 @@ class Context; /** Implements `processes` system table, which allows you to get information about the queries that are currently executing. */ -class StorageSystemProcesses : public ext::shared_ptr_helper, public IStorage +class StorageSystemProcesses : public ext::shared_ptr_helper, public StorageWithVirtualColumns { public: std::string getName() const override { return "SystemProcesses"; } diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 852392a63d1..00dcdc2f051 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -21,95 +22,6 @@ namespace ErrorCodes extern const int CANNOT_GET_CREATE_TABLE_QUERY; } -/// Some virtual columns routines -namespace -{ - -bool hasColumn(const ColumnsWithTypeAndName & columns, const String & column_name) -{ - for (const auto & column : columns) - { - if (column.name == column_name) - return true; - } - - return false; -} - - -NameAndTypePair tryGetColumn(const ColumnsWithTypeAndName & columns, const String & column_name) -{ - for (const auto & column : columns) - { - if (column.name == column_name) - return {column.name, column.type}; - } - - return {}; -} - - -struct VirtualColumnsProcessor -{ - explicit VirtualColumnsProcessor(const ColumnsWithTypeAndName & all_virtual_columns_) - : all_virtual_columns(all_virtual_columns_), virtual_columns_mask(all_virtual_columns_.size(), 0) {} - - /// Separates real and virtual column names, returns real ones - Names process(const Names & column_names, const std::vector & virtual_columns_exists_flag = {}) - { - Names real_column_names; - - if (!virtual_columns_exists_flag.empty()) - { - for (size_t i = 0; i < all_virtual_columns.size(); ++i) - *virtual_columns_exists_flag.at(i) = false; - } - - for (const String & column_name : column_names) - { - ssize_t virtual_column_index = -1; - - for (size_t i = 0; i < all_virtual_columns.size(); ++i) - { - if (column_name == all_virtual_columns[i].name) - { - virtual_column_index = i; - break; - } - } - - if (virtual_column_index >= 0) - { - auto index = static_cast(virtual_column_index); - virtual_columns_mask[index] = 1; - if (!virtual_columns_exists_flag.empty()) - *virtual_columns_exists_flag.at(index) = true; - } - else - { - real_column_names.emplace_back(column_name); - } - } - - return real_column_names; - } - - void appendVirtualColumns(Block & block) - { - for (size_t i = 0; i < all_virtual_columns.size(); ++i) - { - if (virtual_columns_mask[i]) - block.insert(all_virtual_columns[i].cloneEmpty()); - } - } - -protected: - const ColumnsWithTypeAndName & all_virtual_columns; - std::vector virtual_columns_mask; -}; - -} - StorageSystemTables::StorageSystemTables(const std::string & name_) : name(name_) @@ -160,7 +72,7 @@ BlockInputStreams StorageSystemTables::read( bool has_create_table_query = false; bool has_engine_full = false; - VirtualColumnsProcessor virtual_columns_processor(virtual_columns); + auto virtual_columns_processor = getVirtualColumnsProcessor(); real_column_names = virtual_columns_processor.process(column_names, {&has_metadata_modification_time, &has_create_table_query, &has_engine_full}); check(real_column_names); @@ -265,15 +177,4 @@ BlockInputStreams StorageSystemTables::read( return {std::make_shared(res_block)}; } -bool StorageSystemTables::hasColumn(const String & column_name) const -{ - return DB::hasColumn(virtual_columns, column_name) || ITableDeclaration::hasColumn(column_name); -} - -NameAndTypePair StorageSystemTables::getColumn(const String & column_name) const -{ - auto virtual_column = DB::tryGetColumn(virtual_columns, column_name); - return !virtual_column.name.empty() ? virtual_column : ITableDeclaration::getColumn(column_name); -} - } diff --git a/dbms/src/Storages/System/StorageSystemTables.h b/dbms/src/Storages/System/StorageSystemTables.h index fada738af5a..1e1cd4c83aa 100644 --- a/dbms/src/Storages/System/StorageSystemTables.h +++ b/dbms/src/Storages/System/StorageSystemTables.h @@ -2,7 +2,7 @@ #include #include - +#include namespace DB { @@ -12,7 +12,7 @@ class Context; /** Implements the system table `tables`, which allows you to get information about all tables. */ -class StorageSystemTables : public ext::shared_ptr_helper, public IStorage +class StorageSystemTables : public ext::shared_ptr_helper, public StorageWithVirtualColumns { public: std::string getName() const override { return "SystemTables"; } @@ -26,15 +26,9 @@ public: size_t max_block_size, unsigned num_streams) override; - bool hasColumn(const String & column_name) const override; - - NameAndTypePair getColumn(const String & column_name) const override; - private: const std::string name; - ColumnsWithTypeAndName virtual_columns; - protected: StorageSystemTables(const std::string & name_); }; diff --git a/dbms/src/Storages/System/VirtualColumnsProcessor.cpp b/dbms/src/Storages/System/VirtualColumnsProcessor.cpp new file mode 100644 index 00000000000..a126cc3f03a --- /dev/null +++ b/dbms/src/Storages/System/VirtualColumnsProcessor.cpp @@ -0,0 +1,95 @@ +#include "VirtualColumnsProcessor.h" + +namespace DB +{ + +bool hasColumn(const ColumnsWithTypeAndName & columns, const String & column_name) +{ + for (const auto & column : columns) + { + if (column.name == column_name) + return true; + } + + return false; +} + + +NameAndTypePair tryGetColumn(const ColumnsWithTypeAndName & columns, const String & column_name) +{ + for (const auto & column : columns) + { + if (column.name == column_name) + return {column.name, column.type}; + } + + return {}; +} + + +bool StorageWithVirtualColumns::hasColumnImpl(const ITableDeclaration * table, const String & column_name) const +{ + return DB::hasColumn(virtual_columns, column_name) || table->ITableDeclaration::hasColumn(column_name); +} + +NameAndTypePair StorageWithVirtualColumns::getColumnImpl(const ITableDeclaration * table, const String & column_name) const +{ + auto virtual_column = DB::tryGetColumn(virtual_columns, column_name); + return !virtual_column.name.empty() ? virtual_column : table->ITableDeclaration::getColumn(column_name); +} + +VirtualColumnsProcessor StorageWithVirtualColumns::getVirtualColumnsProcessor() +{ + return VirtualColumnsProcessor(virtual_columns); +} + + +Names VirtualColumnsProcessor::process(const Names & column_names, const std::vector & virtual_columns_exists_flag) +{ + Names real_column_names; + + if (!virtual_columns_exists_flag.empty()) + { + for (size_t i = 0; i < all_virtual_columns.size(); ++i) + *virtual_columns_exists_flag.at(i) = false; + } + + for (const String & column_name : column_names) + { + ssize_t virtual_column_index = -1; + + for (size_t i = 0; i < all_virtual_columns.size(); ++i) + { + if (column_name == all_virtual_columns[i].name) + { + virtual_column_index = i; + break; + } + } + + if (virtual_column_index >= 0) + { + auto index = static_cast(virtual_column_index); + virtual_columns_mask[index] = 1; + if (!virtual_columns_exists_flag.empty()) + *virtual_columns_exists_flag.at(index) = true; + } + else + { + real_column_names.emplace_back(column_name); + } + } + + return real_column_names; +} + +void VirtualColumnsProcessor::appendVirtualColumns(Block & block) +{ + for (size_t i = 0; i < all_virtual_columns.size(); ++i) + { + if (virtual_columns_mask[i]) + block.insert(all_virtual_columns[i].cloneEmpty()); + } +} + +} diff --git a/dbms/src/Storages/System/VirtualColumnsProcessor.h b/dbms/src/Storages/System/VirtualColumnsProcessor.h new file mode 100644 index 00000000000..af379bd223b --- /dev/null +++ b/dbms/src/Storages/System/VirtualColumnsProcessor.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include +#include +#include + + +/// Some virtual columns routines +namespace DB +{ + +struct VirtualColumnsProcessor; + + +/// Adaptor for storages having virtual columns +class StorageWithVirtualColumns : public IStorage +{ +public: + + bool hasColumn(const String & column_name) const override + { + return hasColumnImpl(this, column_name); + } + + NameAndTypePair getColumn(const String & column_name) const override + { + return getColumnImpl(this, column_name); + } + +protected: + + VirtualColumnsProcessor getVirtualColumnsProcessor(); + + ColumnsWithTypeAndName virtual_columns; + +protected: + + /// Use these methods instead of regular hasColumn / getColumn + bool hasColumnImpl(const ITableDeclaration * table, const String & column_name) const; + + NameAndTypePair getColumnImpl(const ITableDeclaration * table, const String & column_name) const; +}; + + +struct VirtualColumnsProcessor +{ + explicit VirtualColumnsProcessor(const ColumnsWithTypeAndName & all_virtual_columns_) + : all_virtual_columns(all_virtual_columns_), virtual_columns_mask(all_virtual_columns_.size(), 0) {} + + /// Separates real and virtual column names, returns real ones + Names process(const Names & column_names, const std::vector & virtual_columns_exists_flag = {}); + + /// Append spevified virtual columns (with empty data) to the result block + void appendVirtualColumns(Block & block); + +protected: + const ColumnsWithTypeAndName & all_virtual_columns; + std::vector virtual_columns_mask; +}; + +} diff --git a/dbms/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh b/dbms/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh index d20094b1e61..858016efd91 100755 --- a/dbms/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh +++ b/dbms/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh @@ -25,7 +25,7 @@ function pack_unpack_compare() ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf" ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf_file" - rm -f "${CLICKHOUSE_TMP}/$buf_file" stderr + rm -f "$buf_file" stderr echo $((res_orig - res_db_file)) $((res_orig - res_ch_local1)) $((res_orig - res_ch_local2)) } From b7ee1be6df1db69501198740c8731b3370b72511 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 14 Mar 2018 13:47:13 +0300 Subject: [PATCH 003/315] added ColumnUnique [#CLICKHOUSE-3621] --- dbms/src/Columns/ColumnUnique.h | 50 +++++++++++++++++++ dbms/src/Columns/IColumnUnique.h | 32 ++++++++++++ dbms/src/DataTypes/DataTypeWithDictionary.cpp | 0 dbms/src/DataTypes/DataTypeWithDictionary.h | 34 +++++++++++++ 4 files changed, 116 insertions(+) create mode 100644 dbms/src/Columns/ColumnUnique.h create mode 100644 dbms/src/Columns/IColumnUnique.h create mode 100644 dbms/src/DataTypes/DataTypeWithDictionary.cpp create mode 100644 dbms/src/DataTypes/DataTypeWithDictionary.h diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h new file mode 100644 index 00000000000..75c82e6ab06 --- /dev/null +++ b/dbms/src/Columns/ColumnUnique.h @@ -0,0 +1,50 @@ +#include +#include + +namespace DB +{ + +template +class ColumnUnique : public IColumnUnique +{ +public: + + ColumnPtr getColumn() const overrdie; + size_t insert(const Field & x) overrdie; + ColumnPtr insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + size_t insertData(const char * pos, size_t length) overrdie; + +private: + + struct StringRefWrapper + { + const ColumnType * column = nullptr; + size_t offset = 0; + size_t size = 0; + + StringRefWrapper(const ColumnType * column, size_t row) : column(column) + { + auto ref = column->getDataAt(row); + offset = ref.data - column->getDataAt(0).data; + size = res.size; + } + + operator StringRef() const { return StringRef(column->getDataAt(0).data + offset, size); } + + bool operator== (const StringRefWrapper & other) + { + return (column == other.column && offset == other.offset && size == other.size) + || StringRef(*this) == other; + } + }; + using IndexType = HashMap; + + + MutableColumnPtr column; + /// Lazy initialized. + std::unique_ptr index; + + +}; + +} diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h new file mode 100644 index 00000000000..d4af8d0f6c4 --- /dev/null +++ b/dbms/src/Columns/IColumnUnique.h @@ -0,0 +1,32 @@ +#include + +namespace DB +{ + +class IColumnUnique +{ +public: + /// Column always contains Null if it's Nullable and empty string if it's String or Nullable(String). + /// So, size may be greater than the number of inserted unique values. + virtual ColumnPtr getColumn() const = 0; + virtual size_t size() const { return getColumn()->size(); } + + /// Appends new value at the end of column (column's size is increased by 1). + /// Is used to transform raw strings to Blocks (for example, inside input format parsers) + virtual size_t insert(const Field & x) = 0; + + /// Appends range of elements from other column. + /// Could be used to concatenate columns. + virtual ColumnPtr insertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; + + /// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented). + /// Is used to optimize some computations (in aggregation, for example). + /// Parameter length could be ignored if column values have fixed size. + virtual size_t insertData(const char * pos, size_t length) = 0; + + virtual size_t getInsertionPoint(const char * pos, size_t length) const = 0; + + virtual bool has(const char * pos, size_t length) const { return getInsertionPoint(pos, length) != size(); } +}; + +} diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h new file mode 100644 index 00000000000..631e3e42970 --- /dev/null +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -0,0 +1,34 @@ +#include +#include +#include + +namespace DB +{ + + + +template +class CountingRecursiveDictionary +{ +public: + using DictionaryType = HashMap; + + void insertData(const char * pos, size_t length) { column->insertData(pos, length); } + + StringRef getDataAt(size_t n) const + { + if (n < prev_dictionary_size) + return prev_dictionary->getDataAt(n); + else + return column->getDataAt(n - prev_dictionary_size); + } + +private: + ColumnPtr column; + DictionaryType dictionary; + + std::shared_ptr prev_dictionary; + size_t prev_dictionary_size = 0; +}; + +} From a163459d832cbef5e62513cdd6ff07f1828471c0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Mar 2018 14:45:17 +0300 Subject: [PATCH 004/315] added ColumnWithDictionary [#CLICKHOUSE-3621] --- dbms/src/Columns/ColumnUnique.h | 306 ++++++++++++++++++-- dbms/src/Columns/ColumnWithDictionary.cpp | 24 ++ dbms/src/Columns/ColumnWithDictionary.h | 184 ++++++++++++ dbms/src/Columns/IColumnUnique.h | 57 +++- dbms/src/DataTypes/DataTypeWithDictionary.h | 24 -- 5 files changed, 539 insertions(+), 56 deletions(-) create mode 100644 dbms/src/Columns/ColumnWithDictionary.cpp create mode 100644 dbms/src/Columns/ColumnWithDictionary.h diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 75c82e6ab06..f3750b7c7c8 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -1,50 +1,308 @@ #include #include +#include +#include +#include +#include +#include "ColumnString.h" +class NullMap; namespace DB { -template -class ColumnUnique : public IColumnUnique +template +class ColumnUnique final : public COWPtrHelper { -public: + friend class COWPtrHelper; - ColumnPtr getColumn() const overrdie; - size_t insert(const Field & x) overrdie; - ColumnPtr insertRangeFrom(const IColumn & src, size_t start, size_t length) override; - size_t insertData(const char * pos, size_t length) overrdie; +private: + explicit ColumnUnique(const ColumnPtr & holder); + explicit ColumnUnique(bool is_nullable) : column_holder(ColumnType::create(numSpecialValues())), is_nullable(is_nullable) {} + ColumnUnique(const ColumnUnique & other) : column_holder(other.column_holder), is_nullable(other.is_nullable) {} + +public: + ColumnPtr getNestedColumn() const override { return column_holder; } + size_t uniqueInsert(const Field & x) override; + size_t uniqueInsertFrom(const IColumn & src, size_t n) override; + ColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; + size_t uniqueInsertData(const char * pos, size_t length) override; + size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) override; + size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) override; + + size_t getDefaultValueIndex() const override { return is_nullable ? 1 : 0; } + size_t getNullValueIndex() const override; + bool canContainNulls() const override { return is_nullable; } + + Field operator[](size_t n) const override { return (*column_holder)[n]; } + void get(size_t n, Field & res) const override { column_holder->get(n, res); } + StringRef getDataAt(size_t n) const override { return column_holder->getDataAt(n); } + StringRef getDataAtWithTerminatingZero(size_t n) const override { return column_holder->getDataAtWithTerminatingZero(n); } + UInt64 get64(size_t n) const override { return column_holder->get64(n); } + UInt64 getUInt(size_t n) const override { return column_holder->getUInt(n); } + Int64 getInt(size_t n) const override { return column_holder->getInt(n); } + bool isNullAt(size_t n) const override { return column_holder->isNullAt(n); } + MutableColumnPtr cut(size_t start, size_t length) const override { return column_holder->cut(start, length); } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override { return column_holder->serializeValueIntoArena(n, arena, begin); } + const char * deserializeAndInsertFromArena(const char * pos) override { return column_holder->deserializeAndInsertFromArena(pos); } + void updateHashWithValue(size_t n, SipHash & hash) const override { return column_holder->updateHashWithValue(n, hash); } + MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { return column_holder->filter(filt, result_size_hint); } + MutableColumnPtr permute(const Permutation & perm, size_t limit) const override { return column_holder->permute(perm, limit); } + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override { column_holder->compareAt(n, m, rhs, nan_direction_hint); } + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override { column_holder->getPermutation(reverse, limit, nan_direction_hint, res); } + MutableColumnPtr replicate(const Offsets & offsets) const override { return column_holder->replicate(offsets); } + std::vector scatter(ColumnIndex num_columns, const Selector & selector) const override { return column_holder->scatter(num_columns, selector); } + void getExtremes(Field & min, Field & max) const override { column_holder->getExtremes(min, max); } + bool valuesHaveFixedSize() const override { return column_holder->valuesHaveFixedSize(); } + bool isFixedAndContiguous() const override { return column_holder->isFixedAndContiguous(); } + size_t sizeOfValueIfFixed() const override { return column_holder->sizeOfValueIfFixed(); } + bool isNumeric() const override { return column_holder->isNumeric(); } + + size_t byteSize() const override { return column_holder->byteSize(); } + size_t allocatedBytes() const override { return column_holder->allocatedBytes() + (index ? index->getBufferSizeInBytes() : 0); } + void forEachSubcolumn(ColumnCallback callback) override { callback(column_holder); } private: struct StringRefWrapper { const ColumnType * column = nullptr; - size_t offset = 0; - size_t size = 0; + size_t row = 0; - StringRefWrapper(const ColumnType * column, size_t row) : column(column) + StringRef ref; + + StringRefWrapper(const ColumnType * column, size_t row) : column(column), row(row) {} + StringRefWrapper(StringRef ref) : ref(ref) {} + + operator StringRef() const { return column ? column->getDataAt(row) : ref; } + + bool operator==(const StringRefWrapper & other) { - auto ref = column->getDataAt(row); - offset = ref.data - column->getDataAt(0).data; - size = res.size; - } - - operator StringRef() const { return StringRef(column->getDataAt(0).data + offset, size); } - - bool operator== (const StringRefWrapper & other) - { - return (column == other.column && offset == other.offset && size == other.size) - || StringRef(*this) == other; + return (column && column == other.column && row == other.row) || StringRef(*this) == other; } }; - using IndexType = HashMap; + using IndexMapType = HashMap; - MutableColumnPtr column; + ColumnPtr column_holder; /// Lazy initialized. - std::unique_ptr index; + std::unique_ptr index; + bool is_nullable; + + size_t numSpecialValues() const { return is_nullable ? 2 : 1; } + + void buildIndex(); + ColumnType * getRawColumnPtr() const { return static_cast(column_holder.get()); } + IndexType insert(const StringRefWrapper & ref, IndexType value); }; +template +ColumnUnique::ColumnUnique(const ColumnPtr & holder) : column_holder(holder) +{ + if (column_holder->isColumnNullable()) + { + auto column_nullable = static_cast(column_holder.get()); + column_holder = column_nullable->getNestedColumnPtr(); + is_nullable = true; + } +} + +template +size_t ColumnUnique::getNullValueIndex() const override +{ + if (!is_nullable) + throw Exception("ColumnUnique can't contain null values."); + + return 0; +} + +template +void ColumnUnique::buildIndex() +{ + if (index) + return; + + auto column = getRawColumnPtr(); + index = std::make_unique(); + + for (auto row : ext::range(numSpecialValues(), column->size())) + { + (*index)[StringRefWrapper(column, row)] = row; + } +} + +template +IndexType ColumnUnique::insert(const StringRefWrapper & ref, IndexType value) +{ + if (!index) + buildIndex(); + + IndexType::iterator it; + bool inserted; + index->emplace(ref, it, inserted); + + if (inserted) + it->second = value; + + return it->second; +} + +template +size_t ColumnUnique::uniqueInsert(const Field & x) override +{ + if (x.getType() == Field::Types::Null) + return getNullValueIndex(); + + auto column = getRawColumnPtr(); + IndexType prev_size = static_cast(column->size()); + + if ((*column)[getDefaultValueIndex()] == x) + return getDefaultValueIndex(); + + column->insert(x); + auto pos = insert(StringRefWrapper(column, prev_size), prev_size); + if (pos != prev_size) + column->popBack(1); + + return static_cast(pos); +} + +template +size_t ColumnUnique::uniqueInsertFrom(const IColumn & src, size_t n) override +{ + auto ref = src.getDataAt(n); + return uniqueInsertData(ref.data, ref.size); +} + +template +size_t ColumnUnique::uniqueInsertData(const char * data, size_t length) override +{ + auto column = getRawColumnPtr(); + + if (column->getDataAt(getDefaultValueIndex()) == StringRef(data, length)) + return getDefaultValueIndex(); + + IndexType size = static_cast(column->size()); + + if (!index->has(StringRefWrapper(StringRef(data, length)))) + { + column->insertData(data, length); + return static_cast(insert(StringRefWrapper(StringRef(data, length)), size)); + } + + return size; +} + +template +size_t ColumnUnique::uniqueInsertDataWithTerminatingZero(const char * data, size_t length) override +{ + if (std::is_same::value) + return uniqueInsertData(data, length - 1); + + if (column_holder->valuesHaveFixedSize()) + return uniqueInsertData(data, length); + + /// Don't know if data actually has terminating zero. So, insert it firstly. + + auto column = getRawColumnPtr(); + size_t prev_size = column->size(); + column->insertDataWithTerminatingZero(data, length); + + if (column->compareAt(getDefaultValueIndex(), prev_size, *column, 1) == 0) + { + column->popBack(1); + return getDefaultValueIndex(); + } + + auto pos = insert(StringRefWrapper(column, prev_size), prev_size); + if (pos != prev_size) + column->popBack(1); + + return static_cast(pos); +} + +template +size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) override +{ + auto column = getRawColumnPtr(); + size_t prev_size = column->size(); + new_pos = column->deserializeAndInsertFromArena(pos); + + if (column->compareAt(getDefaultValueIndex(), prev_size, *column, 1) == 0) + { + column->popBack(1); + return getDefaultValueIndex(); + } + + auto index_pos = insert(StringRefWrapper(StringRef(column, prev_size)), prev_size); + if (index_pos != prev_size) + column->popBack(1); + + return static_cast(index_pos); +} + +template +ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) override +{ + if (!index) + buildIndex(); + + const ColumnType * src_column; + const NullMap * null_map = nullptr; + + if (src_column->isNullable()) + { + auto nullable_column = static_cast(&src); + src_column = static_cast(&nullable_column->getNestedColumn()); + null_map = &nullable_column->getNullMapData(); + } + else + src_column = static_cast(&src); + + auto column = getRawColumnPtr(); + IColumn::Filter filter(src_column->size(), 0); + auto positions_column = ColumnVector::create(length); + auto & positions = positions_column.getData(); + + size_t next_position = column->size(); + for (auto i : ext::range(0, length)) + { + auto row = start + i; + + if (column->compareAt(getDefaultValueIndex(), row, *src_column, 1) == 0) + positions[i] = getDefaultValueIndex(); + else if (null_map && (*null_map)[row]) + positions[i] = getNullValueIndex(); + else + { + auto it = index->find(StringRefWrapper(&src_column, row)); + if (it == index->end()) + { + filter[row] = 1; + positions[i] = next_position; + ++next_position; + } + else + positions[i] = it->second; + } + } + + auto filtered_column_ptr = src_column->filter(filter); + auto filtered_column = static_cast(*filtered_column_ptr); + + size_t filtered_size = filtered_column->size(); + + size_t prev_size = column->size(); + column->insertRangeFrom(filtered_column, 0, filtered_size); + + if (filtered_size) + { + for (auto row : ext::range(prev_size, prev_size + filtered_size)) + (*index)[StringRefWrapper(column, row)] = row; + } + + return positions_column; +} + } diff --git a/dbms/src/Columns/ColumnWithDictionary.cpp b/dbms/src/Columns/ColumnWithDictionary.cpp new file mode 100644 index 00000000000..8947e6d7a77 --- /dev/null +++ b/dbms/src/Columns/ColumnWithDictionary.cpp @@ -0,0 +1,24 @@ +#include +#include + +namespace DB +{ + +ColumnWithDictionary::ColumnWithDictionary(const ColumnPtr & column_unique, const ColumnPtr & indexes) + : column_unique(column_unique), indexes(indexes) +{ + if (!dynamic_cast(column_unique.get())) + throw Exception("ColumnUnique expected as argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); +} + +ColumnWithDictionary::ColumnWithDictionary(const ColumnWithDictionary & other) + : column_unique(other.column_unique), indexes(other.indexes) +{ +} + +void ColumnWithDictionary::gather(ColumnGathererStream & gatherer) +{ + gatherer.gather(*this); +} + +} diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h new file mode 100644 index 00000000000..5d963168af2 --- /dev/null +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -0,0 +1,184 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +class ColumnWithDictionary final : public COWPtrHelper +{ + ColumnWithDictionary(const ColumnPtr & column_unique, const ColumnPtr & indexes); + ColumnWithDictionary(const ColumnWithDictionary & other); +public: + std::string getName() const override { return "ColumnWithDictionary"; } + const char * getFamilyName() const override { return "ColumnWithDictionary"; } + + + MutableColumnPtr cloneResized(size_t size) const override + { + return ColumnWithDictionary::create(column_unique, indexes->cloneResized(size)); + } + + size_t size() const override { return indexes->size(); } + + + Field operator[](size_t n) const override { return (*column_unique)[indexes->getUInt(n)]; } + void get(size_t n, Field & res) const override { column_unique->get(indexes->getUInt(n), res); } + + StringRef getDataAt(size_t n) const override { return column_unique->getDataAt(indexes->getUInt(n)); } + + StringRef getDataAtWithTerminatingZero(size_t n) const override + { + return column_unique->getDataAtWithTerminatingZero(indexes->getUInt(n)); + } + + UInt64 get64(size_t n) const override { return column_unique->get64(indexes->getUInt(n)); } + + UInt64 getUInt(size_t n) const override { return column_unique->getUInt(indexes->getUInt(n)); } + Int64 getInt(size_t n) const override { return column_unique->getInt(indexes->getUInt(n)); } + bool isNullAt(size_t n) const override { return column_unique->isNullAt(indexes->getUInt(n)); } + MutableColumnPtr cut(size_t start, size_t length) const override + { + return ColumnWithDictionary::create(column_unique, indexes->cut(start, length)); + } + + void insert(const Field & x) override { indexes->insert(Field(UInt64(getUnique()->uniqueInsert(x)))); } + void insertFrom(const IColumn & src, size_t n) override { indexes->insert(getUnique()->uniqueInsertFrom(src, n)); } + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override + { + auto inserted_indexes = getUnique()->uniqueInsertRangeFrom(src, start, length); + indexes->insertRangeFrom(*inserted_indexes, 0, length); + } + + void insertData(const char * pos, size_t length) override + { + indexes->insert(Field(UInt64(getUnique()->uniqueInsertData(pos, length)))); + } + + void insertDataWithTerminatingZero(const char * pos, size_t length) override + { + indexes->insert(Field(UInt64(getUnique()->uniqueInsertDataWithTerminatingZero(pos, length)))); + } + + void insertDefault() override + { + indexes->insert(getUnique()->getDefaultValueIndex()); + } + + void popBack(size_t n) override { indexes->popBack(n); } + + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override + { + return getUnique()->serializeValueIntoArena(indexes->getUInt(n), arena, begin); + } + + const char * deserializeAndInsertFromArena(const char * pos) override + { + const char * new_pos; + indexes->insert(getUnique()->uniqueDeserializeAndInsertFromArena(pos, new_pos)); + return new_pos; + } + + void updateHashWithValue(size_t n, SipHash & hash) const override + { + return getUnique()->updateHashWithValue(indexes->getUInt(n), hash); + } + + MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override + { + return ColumnWithDictionary::create(column_unique, indexes->filter(filt, result_size_hint)); + } + + MutableColumnPtr permute(const Permutation & perm, size_t limit) const override + { + return ColumnWithDictionary::create(column_unique, indexes->permute(perm, limit)); + } + + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override + { + const auto & column_with_dictionary = static_cast(rhs); + size_t n_index = indexes->getUInt(n); + size_t m_index = column_with_dictionary.indexes->getUInt(m); + return getUnique()->compareAt(n_index, m_index, *column_with_dictionary.column_unique, nan_direction_hint); + } + + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override + { + size_t unique_limit = std::min(limit, getUnique()->size()); + Permutation unique_perm; + getUnique()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); + + /// TODO: optimize with sse. + + /// Get indexes per row in column_unique. + std::vector> indexes_per_row(getUnique()->size()); + size_t indexes_size = indexes->size(); + for (size_t row = 0; row < indexes_size; ++row) + indexes_per_row[indexes->getUInt(row)].push_back(row); + + /// Replicate permutation. + size_t perm_size = std::min(indexes_size, limit); + res.resize(perm_size); + size_t perm_index = 0; + for (size_t row = 0; row < indexes_size && perm_index < perm_size; ++row) + { + const auto & row_indexes = indexes_per_row[unique_perm[row]]; + for (auto row_index : row_indexes) + { + res[perm_index] = row_index; + ++perm_index; + + if (perm_index == perm_size) + break; + } + } + } + + MutableColumnPtr replicate(const Offsets & offsets) const override + { + return ColumnWithDictionary::create(column_unique, indexes->replicate(offsets)); + } + + std::vector scatter(ColumnIndex num_columns, const Selector & selector) const override + { + auto columns = indexes->scatter(num_columns, selector); + for (auto & column : columns) + column = ColumnWithDictionary::create(column_unique, column); + + return columns; + } + + void gather(ColumnGathererStream & gatherer_stream) override ; + void getExtremes(Field & min, Field & max) const override { return column_unique->getExtremes(min, max); } + + void reserve(size_t n) override { indexes->reserve(n); } + + size_t byteSize() const override { return indexes->byteSize() + column_unique->byteSize(); } + size_t allocatedBytes() const override { return indexes->allocatedBytes() + column_unique->allocatedBytes(); } + + void forEachSubcolumn(ColumnCallback callback) override + { + callback(column_unique); + callback(indexes); + } + + bool valuesHaveFixedSize() const override { return column_unique->valuesHaveFixedSize(); } + bool isFixedAndContiguous() const override { return column_unique->isFixedAndContiguous(); } + size_t sizeOfValueIfFixed() const override { return column_unique->sizeOfValueIfFixed(); } + bool isNumeric() const override { return column_unique->isNumeric(); } + +private: + ColumnPtr column_unique; + ColumnPtr indexes; + + IColumnUnique * getUnique() { return static_cast(column_unique->assumeMutable().get()); } + const IColumnUnique * getUnique() const { return static_cast(column_unique->assumeMutable().get()); } +}; + + + +} diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index d4af8d0f6c4..a3c964476cd 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -3,30 +3,71 @@ namespace DB { -class IColumnUnique +class IColumnUnique : public IColumn { public: /// Column always contains Null if it's Nullable and empty string if it's String or Nullable(String). /// So, size may be greater than the number of inserted unique values. - virtual ColumnPtr getColumn() const = 0; - virtual size_t size() const { return getColumn()->size(); } + virtual ColumnPtr getNestedColumn() const = 0; + size_t size() const override { return getNestedColumn()->size(); } /// Appends new value at the end of column (column's size is increased by 1). /// Is used to transform raw strings to Blocks (for example, inside input format parsers) - virtual size_t insert(const Field & x) = 0; + virtual size_t uniqueInsert(const Field & x) = 0; + virtual size_t uniqueInsertFrom(const IColumn & src, size_t n) = 0; /// Appends range of elements from other column. /// Could be used to concatenate columns. - virtual ColumnPtr insertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; + virtual ColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; /// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented). /// Is used to optimize some computations (in aggregation, for example). /// Parameter length could be ignored if column values have fixed size. - virtual size_t insertData(const char * pos, size_t length) = 0; + virtual size_t uniqueInsertData(const char * pos, size_t length) = 0; + virtual size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) = 0; - virtual size_t getInsertionPoint(const char * pos, size_t length) const = 0; + virtual size_t getDefaultValueIndex() const = 0; + virtual size_t getNullValueIndex() const = 0; + virtual bool canContainNulls() const = 0; - virtual bool has(const char * pos, size_t length) const { return getInsertionPoint(pos, length) != size(); } + virtual size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) = 0; + +// virtual size_t getInsertionPoint(const char * pos, size_t length) const = 0; +// +// virtual bool has(const char * pos, size_t length) const { return getInsertionPoint(pos, length) != size(); } + + + const char * getFamilyName() const override { return "ColumnUnique"; } + + void insert(const Field & x) override + { + throw Exception("Method insert is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override + { + throw Exception("Method insertRangeFrom is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + void insertData(const char * pos, size_t length) override + { + throw Exception("Method insertData is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + void insertDefault() override + { + throw Exception("Method insertDefault is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + void popBack() override + { + throw Exception("Method popBack is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + void gather(ColumnGathererStream & gatherer_stream) override + { + throw Exception("Method gather is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } }; } diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index 631e3e42970..b9641d5d764 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -7,28 +7,4 @@ namespace DB -template -class CountingRecursiveDictionary -{ -public: - using DictionaryType = HashMap; - - void insertData(const char * pos, size_t length) { column->insertData(pos, length); } - - StringRef getDataAt(size_t n) const - { - if (n < prev_dictionary_size) - return prev_dictionary->getDataAt(n); - else - return column->getDataAt(n - prev_dictionary_size); - } - -private: - ColumnPtr column; - DictionaryType dictionary; - - std::shared_ptr prev_dictionary; - size_t prev_dictionary_size = 0; -}; - } From 0f0d5b3c0c691172e4a631582a32a8e84436c768 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 29 Mar 2018 17:04:24 +0300 Subject: [PATCH 005/315] added DataTypeWithDictionary [#CLICKHOUSE-3621] --- dbms/src/Columns/ColumnUnique.h | 174 +++++----- dbms/src/Columns/ColumnWithDictionary.cpp | 4 +- dbms/src/Columns/ColumnWithDictionary.h | 54 ++- dbms/src/Columns/IColumnUnique.h | 18 +- dbms/src/DataTypes/DataTypeNullable.h | 2 +- dbms/src/DataTypes/DataTypeWithDictionary.h | 316 +++++++++++++++++- dbms/src/DataTypes/IDataType.cpp | 2 + dbms/src/DataTypes/IDataType.h | 3 + dbms/src/Functions/FunctionsMiscellaneous.cpp | 4 + dbms/src/Functions/FunctionsMiscellaneous.h | 95 ++++++ 10 files changed, 564 insertions(+), 108 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index f3750b7c7c8..ebfa6079685 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -1,27 +1,67 @@ +#pragma once #include #include #include #include #include #include -#include "ColumnString.h" +#include class NullMap; + + +template +struct StringRefWrapper +{ + const ColumnType * column = nullptr; + size_t row = 0; + + StringRef ref; + + StringRefWrapper(const ColumnType * column, size_t row) : column(column), row(row) {} + StringRefWrapper(StringRef ref) : ref(ref) {} + StringRefWrapper(const StringRefWrapper & other) = default; + StringRefWrapper & operator =(int) { column = nullptr; return *this; } + bool operator ==(int) const { return nullptr == column; } + StringRefWrapper() {} + + operator StringRef() const { return column ? column->getDataAt(row) : ref; } + + bool operator==(const StringRefWrapper & other) const + { + return (column && column == other.column && row == other.row) || StringRef(*this) == other; + } + +}; + +namespace ZeroTraits +{ + template + bool check(const StringRefWrapper x) { return nullptr == x.column; } + + template + void set(StringRefWrapper & x) { x.column = nullptr; } +}; + + namespace DB { template -class ColumnUnique final : public COWPtrHelper +class ColumnUnique final : public COWPtrHelper> { - friend class COWPtrHelper; + friend class COWPtrHelper>; private: - explicit ColumnUnique(const ColumnPtr & holder); - explicit ColumnUnique(bool is_nullable) : column_holder(ColumnType::create(numSpecialValues())), is_nullable(is_nullable) {} + explicit ColumnUnique(MutableColumnPtr && holder); + explicit ColumnUnique(const DataTypePtr & type) : is_nullable(type->isNullable()) + { + column_holder = removeNullable(type)->createColumn()->cloneResized(numSpecialValues()); + } ColumnUnique(const ColumnUnique & other) : column_holder(other.column_holder), is_nullable(other.is_nullable) {} public: - ColumnPtr getNestedColumn() const override { return column_holder; } + const ColumnPtr & getNestedColumn() const override { return column_holder; } size_t uniqueInsert(const Field & x) override; size_t uniqueInsertFrom(const IColumn & src, size_t n) override; ColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; @@ -43,14 +83,17 @@ public: bool isNullAt(size_t n) const override { return column_holder->isNullAt(n); } MutableColumnPtr cut(size_t start, size_t length) const override { return column_holder->cut(start, length); } StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override { return column_holder->serializeValueIntoArena(n, arena, begin); } - const char * deserializeAndInsertFromArena(const char * pos) override { return column_holder->deserializeAndInsertFromArena(pos); } void updateHashWithValue(size_t n, SipHash & hash) const override { return column_holder->updateHashWithValue(n, hash); } - MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { return column_holder->filter(filt, result_size_hint); } - MutableColumnPtr permute(const Permutation & perm, size_t limit) const override { return column_holder->permute(perm, limit); } - int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override { column_holder->compareAt(n, m, rhs, nan_direction_hint); } - void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override { column_holder->getPermutation(reverse, limit, nan_direction_hint, res); } - MutableColumnPtr replicate(const Offsets & offsets) const override { return column_holder->replicate(offsets); } - std::vector scatter(ColumnIndex num_columns, const Selector & selector) const override { return column_holder->scatter(num_columns, selector); } + MutableColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override { return column_holder->filter(filt, result_size_hint); } + MutableColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override { return column_holder->permute(perm, limit); } + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override { return column_holder->compareAt(n, m, rhs, nan_direction_hint); } + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override { column_holder->getPermutation(reverse, limit, nan_direction_hint, res); } + MutableColumnPtr replicate(const IColumn::Offsets & offsets) const override + { + auto holder = column_holder; + return std::move(holder)->mutate()->replicate(offsets); + } + std::vector scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const override { return column_holder->scatter(num_columns, selector); } void getExtremes(Field & min, Field & max) const override { column_holder->getExtremes(min, max); } bool valuesHaveFixedSize() const override { return column_holder->valuesHaveFixedSize(); } bool isFixedAndContiguous() const override { return column_holder->isFixedAndContiguous(); } @@ -59,29 +102,11 @@ public: size_t byteSize() const override { return column_holder->byteSize(); } size_t allocatedBytes() const override { return column_holder->allocatedBytes() + (index ? index->getBufferSizeInBytes() : 0); } - void forEachSubcolumn(ColumnCallback callback) override { callback(column_holder); } + void forEachSubcolumn(IColumn::ColumnCallback callback) override { callback(column_holder); } private: - struct StringRefWrapper - { - const ColumnType * column = nullptr; - size_t row = 0; - - StringRef ref; - - StringRefWrapper(const ColumnType * column, size_t row) : column(column), row(row) {} - StringRefWrapper(StringRef ref) : ref(ref) {} - - operator StringRef() const { return column ? column->getDataAt(row) : ref; } - - bool operator==(const StringRefWrapper & other) - { - return (column && column == other.column && row == other.row) || StringRef(*this) == other; - } - }; - - using IndexMapType = HashMap; + using IndexMapType = HashMap, IndexType, StringRefHash>; ColumnPtr column_holder; /// Lazy initialized. @@ -92,13 +117,14 @@ private: size_t numSpecialValues() const { return is_nullable ? 2 : 1; } void buildIndex(); - ColumnType * getRawColumnPtr() const { return static_cast(column_holder.get()); } - IndexType insert(const StringRefWrapper & ref, IndexType value); + ColumnType * getRawColumnPtr() { return static_cast(column_holder->assumeMutable().get()); } + const ColumnType * getRawColumnPtr() const { return static_cast(column_holder.get()); } + IndexType insert(const StringRefWrapper & ref, IndexType value); }; template -ColumnUnique::ColumnUnique(const ColumnPtr & holder) : column_holder(holder) +ColumnUnique::ColumnUnique(MutableColumnPtr && holder) : column_holder(std::move(holder)) { if (column_holder->isColumnNullable()) { @@ -109,7 +135,7 @@ ColumnUnique::ColumnUnique(const ColumnPtr & holder) : column_holder(holder) } template -size_t ColumnUnique::getNullValueIndex() const override +size_t ColumnUnique::getNullValueIndex() const { if (!is_nullable) throw Exception("ColumnUnique can't contain null values."); @@ -118,7 +144,7 @@ size_t ColumnUnique::getNullValueIndex() const override } template -void ColumnUnique::buildIndex() +void ColumnUnique::buildIndex() { if (index) return; @@ -128,17 +154,18 @@ void ColumnUnique::buildIndex() for (auto row : ext::range(numSpecialValues(), column->size())) { - (*index)[StringRefWrapper(column, row)] = row; + (*index)[StringRefWrapper(column, row)] = row; } } template -IndexType ColumnUnique::insert(const StringRefWrapper & ref, IndexType value) +IndexType ColumnUnique::insert(const StringRefWrapper & ref, IndexType value) { if (!index) buildIndex(); - IndexType::iterator it; + using IteratorType = typename IndexMapType::iterator; + IteratorType it; bool inserted; index->emplace(ref, it, inserted); @@ -149,19 +176,19 @@ IndexType ColumnUnique::insert(const StringRefWrapper & ref, IndexType value) } template -size_t ColumnUnique::uniqueInsert(const Field & x) override +size_t ColumnUnique::uniqueInsert(const Field & x) { if (x.getType() == Field::Types::Null) return getNullValueIndex(); auto column = getRawColumnPtr(); - IndexType prev_size = static_cast(column->size()); + auto prev_size = static_cast(column->size()); if ((*column)[getDefaultValueIndex()] == x) return getDefaultValueIndex(); column->insert(x); - auto pos = insert(StringRefWrapper(column, prev_size), prev_size); + auto pos = insert(StringRefWrapper(column, prev_size), prev_size); if (pos != prev_size) column->popBack(1); @@ -169,45 +196,45 @@ size_t ColumnUnique::uniqueInsert(const Field & x) override } template -size_t ColumnUnique::uniqueInsertFrom(const IColumn & src, size_t n) override +size_t ColumnUnique::uniqueInsertFrom(const IColumn & src, size_t n) { auto ref = src.getDataAt(n); return uniqueInsertData(ref.data, ref.size); } template -size_t ColumnUnique::uniqueInsertData(const char * data, size_t length) override +size_t ColumnUnique::uniqueInsertData(const char * pos, size_t length) { auto column = getRawColumnPtr(); - if (column->getDataAt(getDefaultValueIndex()) == StringRef(data, length)) + if (column->getDataAt(getDefaultValueIndex()) == StringRef(pos, length)) return getDefaultValueIndex(); - IndexType size = static_cast(column->size()); + auto size = static_cast(column->size()); - if (!index->has(StringRefWrapper(StringRef(data, length)))) + if (!index->has(StringRefWrapper(StringRef(pos, length)))) { - column->insertData(data, length); - return static_cast(insert(StringRefWrapper(StringRef(data, length)), size)); + column->insertData(pos, length); + return static_cast(insert(StringRefWrapper(StringRef(pos, length)), size)); } return size; } template -size_t ColumnUnique::uniqueInsertDataWithTerminatingZero(const char * data, size_t length) override +size_t ColumnUnique::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) { if (std::is_same::value) - return uniqueInsertData(data, length - 1); + return uniqueInsertData(pos, length - 1); if (column_holder->valuesHaveFixedSize()) - return uniqueInsertData(data, length); + return uniqueInsertData(pos, length); /// Don't know if data actually has terminating zero. So, insert it firstly. auto column = getRawColumnPtr(); size_t prev_size = column->size(); - column->insertDataWithTerminatingZero(data, length); + column->insertDataWithTerminatingZero(pos, length); if (column->compareAt(getDefaultValueIndex(), prev_size, *column, 1) == 0) { @@ -215,15 +242,15 @@ size_t ColumnUnique::uniqueInsertDataWithTerminatingZero(const char * data, size return getDefaultValueIndex(); } - auto pos = insert(StringRefWrapper(column, prev_size), prev_size); - if (pos != prev_size) + auto position = insert(StringRefWrapper(column, prev_size), prev_size); + if (position != prev_size) column->popBack(1); - return static_cast(pos); + return static_cast(position); } template -size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) override +size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) { auto column = getRawColumnPtr(); size_t prev_size = column->size(); @@ -235,7 +262,7 @@ size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(const char * pos, const return getDefaultValueIndex(); } - auto index_pos = insert(StringRefWrapper(StringRef(column, prev_size)), prev_size); + auto index_pos = insert(StringRefWrapper(column, prev_size), prev_size); if (index_pos != prev_size) column->popBack(1); @@ -243,7 +270,7 @@ size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(const char * pos, const } template -ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) override +ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) { if (!index) buildIndex(); @@ -251,7 +278,7 @@ ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, const ColumnType * src_column; const NullMap * null_map = nullptr; - if (src_column->isNullable()) + if (src.isColumnNullable()) { auto nullable_column = static_cast(&src); src_column = static_cast(&nullable_column->getNestedColumn()); @@ -261,9 +288,8 @@ ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, src_column = static_cast(&src); auto column = getRawColumnPtr(); - IColumn::Filter filter(src_column->size(), 0); auto positions_column = ColumnVector::create(length); - auto & positions = positions_column.getData(); + auto & positions = positions_column->getData(); size_t next_position = column->size(); for (auto i : ext::range(0, length)) @@ -276,11 +302,13 @@ ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, positions[i] = getNullValueIndex(); else { - auto it = index->find(StringRefWrapper(&src_column, row)); + auto it = index->find(StringRefWrapper(src_column, row)); if (it == index->end()) { - filter[row] = 1; positions[i] = next_position; + auto ref = src_column->getDataAt(row); + column->insertData(ref.data, ref.size); + (*index)[StringRefWrapper(column, next_position)] = next_position; ++next_position; } else @@ -288,20 +316,6 @@ ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, } } - auto filtered_column_ptr = src_column->filter(filter); - auto filtered_column = static_cast(*filtered_column_ptr); - - size_t filtered_size = filtered_column->size(); - - size_t prev_size = column->size(); - column->insertRangeFrom(filtered_column, 0, filtered_size); - - if (filtered_size) - { - for (auto row : ext::range(prev_size, prev_size + filtered_size)) - (*index)[StringRefWrapper(column, row)] = row; - } - return positions_column; } diff --git a/dbms/src/Columns/ColumnWithDictionary.cpp b/dbms/src/Columns/ColumnWithDictionary.cpp index 8947e6d7a77..6adb07721bf 100644 --- a/dbms/src/Columns/ColumnWithDictionary.cpp +++ b/dbms/src/Columns/ColumnWithDictionary.cpp @@ -4,8 +4,8 @@ namespace DB { -ColumnWithDictionary::ColumnWithDictionary(const ColumnPtr & column_unique, const ColumnPtr & indexes) - : column_unique(column_unique), indexes(indexes) +ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_) + : column_unique(std::move(column_unique_)), indexes(std::move(indexes_)) { if (!dynamic_cast(column_unique.get())) throw Exception("ColumnUnique expected as argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index 5d963168af2..36bf304e8b0 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -1,3 +1,4 @@ +#pragma once #include #include @@ -11,8 +12,11 @@ namespace ErrorCodes class ColumnWithDictionary final : public COWPtrHelper { - ColumnWithDictionary(const ColumnPtr & column_unique, const ColumnPtr & indexes); + friend class COWPtrHelper; + + ColumnWithDictionary(MutableColumnPtr && column_unique, MutableColumnPtr && indexes); ColumnWithDictionary(const ColumnWithDictionary & other); + public: std::string getName() const override { return "ColumnWithDictionary"; } const char * getFamilyName() const override { return "ColumnWithDictionary"; } @@ -20,7 +24,8 @@ public: MutableColumnPtr cloneResized(size_t size) const override { - return ColumnWithDictionary::create(column_unique, indexes->cloneResized(size)); + auto unique_ptr = column_unique; + return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->cloneResized(size)); } size_t size() const override { return indexes->size(); } @@ -43,33 +48,34 @@ public: bool isNullAt(size_t n) const override { return column_unique->isNullAt(indexes->getUInt(n)); } MutableColumnPtr cut(size_t start, size_t length) const override { - return ColumnWithDictionary::create(column_unique, indexes->cut(start, length)); + auto unique_ptr = column_unique; + return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->cut(start, length)); } - void insert(const Field & x) override { indexes->insert(Field(UInt64(getUnique()->uniqueInsert(x)))); } - void insertFrom(const IColumn & src, size_t n) override { indexes->insert(getUnique()->uniqueInsertFrom(src, n)); } + void insert(const Field & x) override { getIndexes()->insert(Field(UInt64(getUnique()->uniqueInsert(x)))); } + void insertFrom(const IColumn & src, size_t n) override { getIndexes()->insert(getUnique()->uniqueInsertFrom(src, n)); } void insertRangeFrom(const IColumn & src, size_t start, size_t length) override { auto inserted_indexes = getUnique()->uniqueInsertRangeFrom(src, start, length); - indexes->insertRangeFrom(*inserted_indexes, 0, length); + getIndexes()->insertRangeFrom(*inserted_indexes, 0, length); } void insertData(const char * pos, size_t length) override { - indexes->insert(Field(UInt64(getUnique()->uniqueInsertData(pos, length)))); + getIndexes()->insert(Field(UInt64(getUnique()->uniqueInsertData(pos, length)))); } void insertDataWithTerminatingZero(const char * pos, size_t length) override { - indexes->insert(Field(UInt64(getUnique()->uniqueInsertDataWithTerminatingZero(pos, length)))); + getIndexes()->insert(Field(UInt64(getUnique()->uniqueInsertDataWithTerminatingZero(pos, length)))); } void insertDefault() override { - indexes->insert(getUnique()->getDefaultValueIndex()); + getIndexes()->insert(getUnique()->getDefaultValueIndex()); } - void popBack(size_t n) override { indexes->popBack(n); } + void popBack(size_t n) override { getIndexes()->popBack(n); } StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override { @@ -79,7 +85,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override { const char * new_pos; - indexes->insert(getUnique()->uniqueDeserializeAndInsertFromArena(pos, new_pos)); + getIndexes()->insert(getUnique()->uniqueDeserializeAndInsertFromArena(pos, new_pos)); return new_pos; } @@ -90,12 +96,14 @@ public: MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { - return ColumnWithDictionary::create(column_unique, indexes->filter(filt, result_size_hint)); + auto unique_ptr = column_unique; + return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->filter(filt, result_size_hint)); } MutableColumnPtr permute(const Permutation & perm, size_t limit) const override { - return ColumnWithDictionary::create(column_unique, indexes->permute(perm, limit)); + auto unique_ptr = column_unique; + return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->permute(perm, limit)); } int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override @@ -140,14 +148,18 @@ public: MutableColumnPtr replicate(const Offsets & offsets) const override { - return ColumnWithDictionary::create(column_unique, indexes->replicate(offsets)); + auto unique_ptr = column_unique; + return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->replicate(offsets)); } std::vector scatter(ColumnIndex num_columns, const Selector & selector) const override { auto columns = indexes->scatter(num_columns, selector); for (auto & column : columns) - column = ColumnWithDictionary::create(column_unique, column); + { + auto unique_ptr = column_unique; + column = ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), std::move(column)); + } return columns; } @@ -155,7 +167,7 @@ public: void gather(ColumnGathererStream & gatherer_stream) override ; void getExtremes(Field & min, Field & max) const override { return column_unique->getExtremes(min, max); } - void reserve(size_t n) override { indexes->reserve(n); } + void reserve(size_t n) override { getIndexes()->reserve(n); } size_t byteSize() const override { return indexes->byteSize() + column_unique->byteSize(); } size_t allocatedBytes() const override { return indexes->allocatedBytes() + column_unique->allocatedBytes(); } @@ -171,12 +183,18 @@ public: size_t sizeOfValueIfFixed() const override { return column_unique->sizeOfValueIfFixed(); } bool isNumeric() const override { return column_unique->isNumeric(); } + IColumnUnique * getUnique() { return static_cast(column_unique->assumeMutable().get()); } + const IColumnUnique * getUnique() const { return static_cast(column_unique->assumeMutable().get()); } + const ColumnPtr & getUniquePtr() const { return column_unique; } + + IColumn * getIndexes() { return indexes->assumeMutable().get(); } + const IColumn * getIndexes() const { return indexes.get(); } + const ColumnPtr & getIndexesPtr() const { return indexes; } + private: ColumnPtr column_unique; ColumnPtr indexes; - IColumnUnique * getUnique() { return static_cast(column_unique->assumeMutable().get()); } - const IColumnUnique * getUnique() const { return static_cast(column_unique->assumeMutable().get()); } }; diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index a3c964476cd..e136c12c697 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -1,3 +1,4 @@ +#pragma once #include namespace DB @@ -8,7 +9,7 @@ class IColumnUnique : public IColumn public: /// Column always contains Null if it's Nullable and empty string if it's String or Nullable(String). /// So, size may be greater than the number of inserted unique values. - virtual ColumnPtr getNestedColumn() const = 0; + virtual const ColumnPtr & getNestedColumn() const = 0; size_t size() const override { return getNestedColumn()->size(); } /// Appends new value at the end of column (column's size is increased by 1). @@ -39,17 +40,17 @@ public: const char * getFamilyName() const override { return "ColumnUnique"; } - void insert(const Field & x) override + void insert(const Field &) override { throw Exception("Method insert is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override + void insertRangeFrom(const IColumn &, size_t, size_t) override { throw Exception("Method insertRangeFrom is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } - void insertData(const char * pos, size_t length) override + void insertData(const char *, size_t) override { throw Exception("Method insertData is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } @@ -59,15 +60,20 @@ public: throw Exception("Method insertDefault is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } - void popBack() override + void popBack(size_t) override { throw Exception("Method popBack is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } - void gather(ColumnGathererStream & gatherer_stream) override + void gather(ColumnGathererStream &) override { throw Exception("Method gather is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } + + const char * deserializeAndInsertFromArena(const char *) override + { + throw Exception("Method deserializeAndInsertFromArena is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } }; } diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index 33fe97fc7c0..5a3be2c4c05 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -13,7 +13,7 @@ class DataTypeNullable final : public IDataType public: static constexpr bool is_parametric = true; - DataTypeNullable(const DataTypePtr & nested_data_type_); + explicit DataTypeNullable(const DataTypePtr & nested_data_type_); std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index b9641d5d764..f5a9a89234f 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -1,10 +1,324 @@ -#include +#pragma once +#include +#include #include #include +#include +#include +#include +#include +#include +#include "DataTypeDate.h" +#include "DataTypeDateTime.h" namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} +class DataTypeWithDictionary : public IDataType +{ +private: + DataTypePtr dictionary_type; + DataTypePtr indexes_type; + +public: + + DataTypeWithDictionary(DataTypePtr dictionary_type_, DataTypePtr indexes_type_) + : dictionary_type(std::move(dictionary_type_)), indexes_type(std::move(indexes_type_)) + { + if (!indexes_type->isUnsignedInteger()) + throw Exception("Index type of DataTypeWithDictionary must be unsigned integer, but got " + + indexes_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto inner_type = dictionary_type; + if (dictionary_type->isNullable()) + inner_type = static_cast(*dictionary_type).getNestedType(); + + if (!inner_type->isStringOrFixedString() + && !inner_type->isDateOrDateTime() + && !inner_type->isNumber()) + throw Exception("DataTypeWithDictionary is supported only for numbers, strings, Date or DateTime, but got " + + dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + const DataTypePtr & getDictionaryType() const { return dictionary_type; } + const DataTypePtr & getIndexesType() const { return indexes_type; } + + String getName() const override + { + return "WithDictionary(" + dictionary_type->getName() + ", " + indexes_type->getName() + ")"; + } + const char * getFamilyName() const override { return "WithDictionary"; } + + void enumerateStreams(StreamCallback callback, SubstreamPath path) const override + { + path.push_back(Substream::DictionaryElements); + dictionary_type->enumerateStreams(callback, path); + path.back() = Substream::DictionaryIndexes; + indexes_type->enumerateStreams(callback, path); + } + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + OutputStreamGetter getter, + size_t offset, + size_t limit, + bool /*position_independent_encoding*/, + SubstreamPath path) const override + { + const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + + path.push_back(Substream::DictionaryElements); + if (auto stream = getter(path)) + dictionary_type->serializeBinaryBulk(*column_with_dictionary.getUnique(), *stream, offset, limit); + + path.back() = Substream::DictionaryIndexes; + if (auto stream = getter(path)) + indexes_type->serializeBinaryBulk(*column_with_dictionary.getIndexes(), *stream, offset, limit); + } + + void deserializeBinaryBulkWithMultipleStreams( + IColumn & column, + InputStreamGetter getter, + size_t limit, + double /*avg_value_size_hint*/, + bool /*position_independent_encoding*/, + SubstreamPath path) const override + { + ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + + path.push_back(Substream::DictionaryElements); + if (ReadBuffer * stream = getter(path)) + dictionary_type->deserializeBinaryBulk(*column_with_dictionary.getUnique(), *stream, limit, 0); + + path.back() = Substream::DictionaryIndexes; + if (auto stream = getter(path)) + indexes_type->deserializeBinaryBulk(*column_with_dictionary.getIndexes(), *stream, limit, 0); + } + + void serializeBinary(const Field & field, WriteBuffer & ostr) const override { dictionary_type->serializeBinary(field, ostr); } + void deserializeBinary(Field & field, ReadBuffer & istr) const override { dictionary_type->deserializeBinary(field, istr); } + + const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column) const + { + return typeid_cast(column);; + } + + ColumnWithDictionary & getColumnWithDictionary(IColumn & column) const + { + return typeid_cast(column);; + } + + IColumn & getNestedUniqueColumn(ColumnWithDictionary & column_with_dictionary) const + { + return *column_with_dictionary.getUnique()->getNestedColumn()->assumeMutable(); + } + + template + using SerealizeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, WriteBuffer &, Args & ...) const; + + template + void serializeImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, SerealizeFunctionPtr func, Args & ... args) const + { + auto & column_with_dictionary = getColumnWithDictionary(column); + size_t unique_row_number = column_with_dictionary.getIndexes()->getUInt(row_num); + (dictionary_type.get()->*func)(*column_with_dictionary.getUnique(), unique_row_number, ostr, std::forward(args)...); + } + + template + using DeserealizeFunctionPtr = void (IDataType::*)(IColumn &, ReadBuffer &, Args ...) const; + + template + void deserializeImpl(IColumn & column, ReadBuffer & istr, DeserealizeFunctionPtr func, Args ... args) const + { + auto & column_with_dictionary = getColumnWithDictionary(column); + auto nested_unique = getNestedUniqueColumn(column_with_dictionary).assumeMutable(); + + auto size = column_with_dictionary.size(); + auto unique_size = nested_unique->size(); + + (dictionary_type.get()->*func)(*nested_unique, istr, std::forward(args)...); + + /// Note: Insertion into ColumnWithDictionary from it's nested column may cause insertion from column to itself. + /// Generally it's wrong because column may reallocate memory before insertion. + column_with_dictionary.insertFrom(*nested_unique, unique_size); + if (column_with_dictionary.getIndexes()->getUInt(size) != unique_size) + nested_unique->popBack(1); + } + + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override + { + serializeImpl(column, row_num, ostr, &IDataType::serializeBinary); + } + void deserializeBinary(IColumn & column, ReadBuffer & istr) const override + { + deserializeImpl(column, istr, &IDataType::deserializeBinary); + } + + void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override + { + serializeImpl(column, row_num, ostr, &IDataType::serializeTextEscaped); + } + + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const override + { + deserializeImpl(column, istr, &IDataType::deserializeTextEscaped); + } + + void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override + { + serializeImpl(column, row_num, ostr, &IDataType::serializeTextQuoted); + } + + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const override + { + deserializeImpl(column, istr, &IDataType::deserializeTextQuoted); + } + + void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override + { + serializeImpl(column, row_num, ostr, &IDataType::serializeTextCSV); + } + + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override + { + deserializeImpl(column, istr, &IDataType::deserializeTextCSV, delimiter); + } + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override + { + serializeImpl(column, row_num, ostr, &IDataType::serializeText); + } + + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettingsJSON & settings) const override + { + serializeImpl(column, row_num, ostr, &IDataType::serializeTextJSON, settings); + } + void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const override + { + deserializeImpl(column, istr, &IDataType::deserializeTextJSON); + } + + void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override + { + serializeImpl(column, row_num, ostr, &IDataType::serializeTextXML); + } + + template + MutableColumnPtr createColumnImpl() const + { + return ColumnWithDictionary::create(ColumnUnique::create(dictionary_type), + indexes_type->createColumn()); + } + + template + MutableColumnPtr createColumnImpl() const + { + if (typeid_cast(indexes_type.get())) + return createColumnImpl(); + if (typeid_cast(indexes_type.get())) + return createColumnImpl(); + if (typeid_cast(indexes_type.get())) + return createColumnImpl(); + if (typeid_cast(indexes_type.get())) + return createColumnImpl(); + + throw Exception("The type of indexes must be unsigned integer, but got " + dictionary_type->getName(), + ErrorCodes::LOGICAL_ERROR); + } + +private: + struct CreateColumnVector + { + MutableColumnPtr & column; + const DataTypeWithDictionary * data_type_with_dictionary; + const IDataType * type; + + CreateColumnVector(MutableColumnPtr & column, const DataTypeWithDictionary * data_type_with_dictionary, + const IDataType * type) + : column(column), data_type_with_dictionary(data_type_with_dictionary), type(type) {} + + template + void operator()() + { + if (typeid_cast *>(type)) + column = data_type_with_dictionary->createColumnImpl>(); + } + }; + +public: + MutableColumnPtr createColumn() const override + { + auto type = dictionary_type; + if (type->isNullable()) + type = static_cast(*dictionary_type).getNestedType(); + + if (type->isString()) + return createColumnImpl(); + if (type->isFixedString()) + return createColumnImpl(); + if (typeid_cast(type.get())) + return createColumnImpl>(); + if (typeid_cast(type.get())) + return createColumnImpl>(); + if (type->isNumber()) + { + MutableColumnPtr column; + TypeListNumbers::forEach(CreateColumnVector(column, this, dictionary_type.get())); + + if (!column) + throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR); + + return std::move(column); + } + + throw Exception("Unexpected dictionary type for DataTypeWithDictionary: " + type->getName(), ErrorCodes::LOGICAL_ERROR); + } + + Field getDefault() const override { return dictionary_type->getDefault(); } + + bool equals(const IDataType & rhs) const override + { + if (typeid(rhs) != typeid(*this)) + return false; + + auto & rhs_with_dictionary = static_cast(rhs); + return dictionary_type->equals(*rhs_with_dictionary.dictionary_type) + && indexes_type->equals(*rhs_with_dictionary.indexes_type); + } + + bool isParametric() const override { return true; } + bool haveSubtypes() const override { return true; } + bool cannotBeStoredInTables() const override { return dictionary_type->cannotBeStoredInTables(); } + bool shouldAlignRightInPrettyFormats() const override { return dictionary_type->shouldAlignRightInPrettyFormats(); } + bool textCanContainOnlyValidUTF8() const override { return dictionary_type->textCanContainOnlyValidUTF8(); } + bool isComparable() const override { return dictionary_type->isComparable(); } + bool canBeComparedWithCollation() const override { return dictionary_type->canBeComparedWithCollation(); } + bool canBeUsedAsVersion() const override { return dictionary_type->canBeUsedAsVersion(); } + bool isSummable() const override { return dictionary_type->isSummable(); }; + bool canBeUsedInBitOperations() const override { return dictionary_type->canBeUsedInBitOperations(); }; + bool canBeUsedInBooleanContext() const override { return dictionary_type->canBeUsedInBooleanContext(); }; + bool isNumber() const override { return false; } + bool isInteger() const override { return false; } + bool isUnsignedInteger() const override { return false; } + bool isDateOrDateTime() const override { return false; } + bool isValueRepresentedByNumber() const override { return dictionary_type->isValueRepresentedByNumber(); } + bool isValueRepresentedByInteger() const override { return dictionary_type->isValueRepresentedByInteger(); } + bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } + bool isString() const override { return false; } + bool isFixedString() const override { return false; } + bool haveMaximumSizeOfValue() const override { return dictionary_type->haveMaximumSizeOfValue(); } + size_t getMaximumSizeOfValueInMemory() const override { return dictionary_type->getMaximumSizeOfValueInMemory(); } + size_t getSizeOfValueInMemory() const override { return dictionary_type->getSizeOfValueInMemory(); } + bool isCategorial() const override { return false; } + bool isEnum() const override { return false; } + bool isNullable() const override { return false; } + bool onlyNull() const override { return false; } +}; } diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index 68fe74d18f0..fe092268b42 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -92,6 +92,8 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy /// and name is encoded as a whole. stream_name += "%2E" + escapeForFileName(elem.tuple_element_name); } + else if (elem.type == Substream::DictionaryElements) + stream_name += ".dict"; } return stream_name; } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 3fd3bc7f005..943e9c9e69f 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -79,6 +79,9 @@ public: NullMap, TupleElement, + + DictionaryElements, + DictionaryIndexes, }; Type type; diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index 99ed8b2dda6..7f515df76ea 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -1876,5 +1876,9 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Functions/FunctionsMiscellaneous.h b/dbms/src/Functions/FunctionsMiscellaneous.h index fdec1fe3fdb..78184e81105 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.h +++ b/dbms/src/Functions/FunctionsMiscellaneous.h @@ -6,6 +6,8 @@ #include #include #include +#include +#include namespace DB { @@ -35,6 +37,99 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; }; +class FunctionMakeDictionary: public IFunction +{ +public: + static constexpr auto name = "makeDictionary"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return std::make_shared(arguments[0], std::make_shared()); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + auto column = res.type->createColumn(); + column->insertRangeFrom(*arg.column, 0, arg.column->size()); + res.column = std::move(column); + } +}; + +class FunctionDictionaryIndexes: public IFunction +{ +public: + static constexpr auto name = "dictionaryIndexes"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + auto * type = typeid_cast(arguments[0].get()); + if (!type) + throw Exception("First first argument of function dictionaryIndexes must be ColumnWithDictionary, but got" + + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getIndexesType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + res.column = typeid_cast(arg.column.get())->getIndexesPtr(); + } +}; + +class FunctionDictionaryValues: public IFunction +{ +public: + static constexpr auto name = "dictionaryValues"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + auto * type = typeid_cast(arguments[0].get()); + if (!type) + throw Exception("First first argument of function dictionaryValues must be ColumnWithDictionary, but got" + + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getDictionaryType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + const auto * column_with_dictionary = typeid_cast(arg.column.get()); + res.column = column_with_dictionary->getUnique()->getNestedColumn()->cloneResized(arg.column->size()); + } +}; /// Executes expression. Uses for lambda functions implementation. Can't be created from factory. class FunctionExpression : public IFunctionBase, public IPreparedFunction, From b8957e4d970fbf1b2ecfa4716d6f5cc3dbfccea7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Apr 2018 13:45:05 +0300 Subject: [PATCH 006/315] fix build --- dbms/src/Columns/ColumnUnique.h | 49 ++++++++++++++++++------- dbms/src/Columns/ColumnWithDictionary.h | 35 +++++++++++------- 2 files changed, 58 insertions(+), 26 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index ebfa6079685..0aa3ff4ede4 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -76,24 +76,44 @@ public: Field operator[](size_t n) const override { return (*column_holder)[n]; } void get(size_t n, Field & res) const override { column_holder->get(n, res); } StringRef getDataAt(size_t n) const override { return column_holder->getDataAt(n); } - StringRef getDataAtWithTerminatingZero(size_t n) const override { return column_holder->getDataAtWithTerminatingZero(n); } + StringRef getDataAtWithTerminatingZero(size_t n) const override + { + return column_holder->getDataAtWithTerminatingZero(n); + } UInt64 get64(size_t n) const override { return column_holder->get64(n); } UInt64 getUInt(size_t n) const override { return column_holder->getUInt(n); } Int64 getInt(size_t n) const override { return column_holder->getInt(n); } bool isNullAt(size_t n) const override { return column_holder->isNullAt(n); } - MutableColumnPtr cut(size_t start, size_t length) const override { return column_holder->cut(start, length); } - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override { return column_holder->serializeValueIntoArena(n, arena, begin); } - void updateHashWithValue(size_t n, SipHash & hash) const override { return column_holder->updateHashWithValue(n, hash); } - MutableColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override { return column_holder->filter(filt, result_size_hint); } - MutableColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override { return column_holder->permute(perm, limit); } - int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override { return column_holder->compareAt(n, m, rhs, nan_direction_hint); } - void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override { column_holder->getPermutation(reverse, limit, nan_direction_hint, res); } - MutableColumnPtr replicate(const IColumn::Offsets & offsets) const override + ColumnPtr cut(size_t start, size_t length) const override { return column_holder->cut(start, length); } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override { - auto holder = column_holder; - return std::move(holder)->mutate()->replicate(offsets); + return column_holder->serializeValueIntoArena(n, arena, begin); + } + void updateHashWithValue(size_t n, SipHash & hash) const override + { + return column_holder->updateHashWithValue(n, hash); + } + ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override + { + return column_holder->filter(filt, result_size_hint); + } + ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override + { + return column_holder->permute(perm, limit); + } + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override + { + return column_holder->compareAt(n, m, rhs, nan_direction_hint); + } + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override + { + column_holder->getPermutation(reverse, limit, nan_direction_hint, res); + } + ColumnPtr replicate(const IColumn::Offsets & offsets) const override { return column_holder->replicate(offsets); } + std::vector scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const override + { + return column_holder->scatter(num_columns, selector); } - std::vector scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const override { return column_holder->scatter(num_columns, selector); } void getExtremes(Field & min, Field & max) const override { column_holder->getExtremes(min, max); } bool valuesHaveFixedSize() const override { return column_holder->valuesHaveFixedSize(); } bool isFixedAndContiguous() const override { return column_holder->isFixedAndContiguous(); } @@ -101,7 +121,10 @@ public: bool isNumeric() const override { return column_holder->isNumeric(); } size_t byteSize() const override { return column_holder->byteSize(); } - size_t allocatedBytes() const override { return column_holder->allocatedBytes() + (index ? index->getBufferSizeInBytes() : 0); } + size_t allocatedBytes() const override + { + return column_holder->allocatedBytes() + (index ? index->getBufferSizeInBytes() : 0); + } void forEachSubcolumn(IColumn::ColumnCallback callback) override { callback(column_holder); } private: diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index 36bf304e8b0..7d8a216b858 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -18,6 +18,19 @@ class ColumnWithDictionary final : public COWPtrHelper; + static Ptr create(const ColumnPtr & column_unique_, const ColumnPtr & indexes_) + { + return ColumnWithDictionary::create(column_unique_->assumeMutable(), indexes_->assumeMutable()); + } + + template ::value>::type> + static MutablePtr create(Args &&... args) { return Base::create(std::forward(args)...); } + + std::string getName() const override { return "ColumnWithDictionary"; } const char * getFamilyName() const override { return "ColumnWithDictionary"; } @@ -46,10 +59,9 @@ public: UInt64 getUInt(size_t n) const override { return column_unique->getUInt(indexes->getUInt(n)); } Int64 getInt(size_t n) const override { return column_unique->getInt(indexes->getUInt(n)); } bool isNullAt(size_t n) const override { return column_unique->isNullAt(indexes->getUInt(n)); } - MutableColumnPtr cut(size_t start, size_t length) const override + ColumnPtr cut(size_t start, size_t length) const override { - auto unique_ptr = column_unique; - return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->cut(start, length)); + return ColumnWithDictionary::create(column_unique, indexes->cut(start, length)); } void insert(const Field & x) override { getIndexes()->insert(Field(UInt64(getUnique()->uniqueInsert(x)))); } @@ -94,16 +106,14 @@ public: return getUnique()->updateHashWithValue(indexes->getUInt(n), hash); } - MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override + ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { - auto unique_ptr = column_unique; - return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->filter(filt, result_size_hint)); + return ColumnWithDictionary::create(column_unique, indexes->filter(filt, result_size_hint)); } - MutableColumnPtr permute(const Permutation & perm, size_t limit) const override + ColumnPtr permute(const Permutation & perm, size_t limit) const override { - auto unique_ptr = column_unique; - return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->permute(perm, limit)); + return ColumnWithDictionary::create(column_unique, indexes->permute(perm, limit)); } int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override @@ -146,10 +156,9 @@ public: } } - MutableColumnPtr replicate(const Offsets & offsets) const override + ColumnPtr replicate(const Offsets & offsets) const override { - auto unique_ptr = column_unique; - return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->replicate(offsets)); + return ColumnWithDictionary::create(column_unique, indexes->replicate(offsets)); } std::vector scatter(ColumnIndex num_columns, const Selector & selector) const override @@ -158,7 +167,7 @@ public: for (auto & column : columns) { auto unique_ptr = column_unique; - column = ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), std::move(column)); + column = ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), std::move(column)); } return columns; From 7f9454cc77176059b49dc5047c21698ab943ef64 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Apr 2018 14:12:11 +0300 Subject: [PATCH 007/315] fix build --- dbms/src/Columns/ColumnWithDictionary.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index 7d8a216b858..1f65a4d651a 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -38,7 +38,7 @@ public: MutableColumnPtr cloneResized(size_t size) const override { auto unique_ptr = column_unique; - return ColumnWithDictionary::create(std::move(unique_ptr)->mutate(), indexes->cloneResized(size)); + return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), indexes->cloneResized(size)); } size_t size() const override { return indexes->size(); } From 852992ac3bbf5ec15b0eff9dd5ec157eb65ebec2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Apr 2018 14:22:35 +0300 Subject: [PATCH 008/315] fix build --- dbms/src/DataTypes/DataTypeWithDictionary.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index f5a9a89234f..e7820b9f0a7 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -116,7 +116,7 @@ public: IColumn & getNestedUniqueColumn(ColumnWithDictionary & column_with_dictionary) const { - return *column_with_dictionary.getUnique()->getNestedColumn()->assumeMutable(); + return column_with_dictionary.getUnique()->getNestedColumn()->assumeMutableRef(); } template From d97e7362dbc476d73bce0671186a99821e3dcd2f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Apr 2018 14:28:35 +0300 Subject: [PATCH 009/315] fix build --- dbms/src/Columns/ColumnUnique.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 0aa3ff4ede4..8fa56eb94b9 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -6,6 +6,7 @@ #include #include #include +#include class NullMap; From cdc8bbea7edef9c7a6be2af2cb5961235997c11e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Apr 2018 15:41:16 +0300 Subject: [PATCH 010/315] fix build --- dbms/src/Columns/ColumnUnique.h | 50 ++++++++++++++++++++++++++------- 1 file changed, 40 insertions(+), 10 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 8fa56eb94b9..df5d94d905a 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -55,14 +55,11 @@ class ColumnUnique final : public COWPtrHelperisNullable()) - { - column_holder = removeNullable(type)->createColumn()->cloneResized(numSpecialValues()); - } + explicit ColumnUnique(const DataTypePtr & type); ColumnUnique(const ColumnUnique & other) : column_holder(other.column_holder), is_nullable(other.is_nullable) {} public: - const ColumnPtr & getNestedColumn() const override { return column_holder; } + const ColumnPtr & getNestedColumn() const override; size_t uniqueInsert(const Field & x) override; size_t uniqueInsertFrom(const IColumn & src, size_t n) override; ColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; @@ -133,6 +130,11 @@ private: using IndexMapType = HashMap, IndexType, StringRefHash>; ColumnPtr column_holder; + + /// For DataTypeNullable, nullptr otherwise. + ColumnPtr nullable_column; + NullMap * nullable_column_map = nullptr; + /// Lazy initialized. std::unique_ptr index; @@ -147,17 +149,45 @@ private: }; +template +ColumnUnique::ColumnUnique(const DataTypePtr & type) : is_nullable(type->isNullable()) +{ + if (is_nullable) + { + nullable_column = type->createColumn()->cloneResized(numSpecialValues()); + auto & column_nullable = static_cast(nullable_column->assumeMutableRef()); + column_holder = column_nullable.getNestedColumnPtr(); + nullable_column_map = &column_nullable.getNullMapData(); + (*nullable_column_map)[1] = 0; + } + else + column_holder = type->createColumn()->cloneResized(numSpecialValues()); +} + template ColumnUnique::ColumnUnique(MutableColumnPtr && holder) : column_holder(std::move(holder)) { if (column_holder->isColumnNullable()) { - auto column_nullable = static_cast(column_holder.get()); - column_holder = column_nullable->getNestedColumnPtr(); + nullable_column = std::move(column_holder); + auto & column_nullable = static_cast(nullable_column->assumeMutableRef()); + column_holder = column_nullable.getNestedColumnPtr(); + nullable_column_map = &column_nullable.getNullMapData(); is_nullable = true; } } +template +const ColumnPtr& ColumnUnique::getNestedColumn() const +{ + if (is_nullable) + { + nullable_column_map->resize_fill(column_holder->size()); + return nullable_column; + } + return column_holder; +} + template size_t ColumnUnique::getNullValueIndex() const { @@ -320,10 +350,10 @@ ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColu { auto row = start + i; - if (column->compareAt(getDefaultValueIndex(), row, *src_column, 1) == 0) - positions[i] = getDefaultValueIndex(); - else if (null_map && (*null_map)[row]) + if (null_map && (*null_map)[row]) positions[i] = getNullValueIndex(); + else if (column->compareAt(getDefaultValueIndex(), row, *src_column, 1) == 0) + positions[i] = getDefaultValueIndex(); else { auto it = index->find(StringRefWrapper(src_column, row)); From e9bc0718376ebdefce581b69310059a634eb0d7b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Apr 2018 20:47:27 +0300 Subject: [PATCH 011/315] default implementation for functions with ColumnWithDictionary arguments --- dbms/src/Columns/ColumnWithDictionary.h | 21 ++++ dbms/src/Columns/IColumn.h | 2 + dbms/src/DataTypes/DataTypeWithDictionary.h | 1 + dbms/src/DataTypes/IDataType.h | 38 +++--- dbms/src/Functions/IFunction.cpp | 126 +++++++++++++++++++- dbms/src/Functions/IFunction.h | 17 +++ 6 files changed, 183 insertions(+), 22 deletions(-) diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index 1f65a4d651a..c9b2fffce1c 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -1,6 +1,9 @@ #pragma once #include #include +#include +#include +#include "ColumnsNumber.h" namespace DB { @@ -34,6 +37,22 @@ public: std::string getName() const override { return "ColumnWithDictionary"; } const char * getFamilyName() const override { return "ColumnWithDictionary"; } + ColumnPtr convertToFullColumn() const + { + auto & nested = getUnique()->getNestedColumn(); + auto * indexes = getIndexes(); + if (auto * column_uint8 = typeid_cast(indexes)) + return nested->sample(column_uint8->getData()); + else if (auto * column_uint16 = typeid_cast(indexes)) + return nested->sample(column_uint16->getData()); + else if (auto * column_uint32 = typeid_cast(indexes)) + return nested->sample(column_uint32->getData()); + else if (auto * column_uint64 = typeid_cast(indexes)) + return nested->sample(column_uint64->getData()); + else + throw Exception("Indexes column for ColumnWithDictionary expected to be ColumnUInt, got " + + indexes->getName(), ErrorCodes::LOGICAL_ERROR); + } MutableColumnPtr cloneResized(size_t size) const override { @@ -200,6 +219,8 @@ public: const IColumn * getIndexes() const { return indexes.get(); } const ColumnPtr & getIndexesPtr() const { return indexes; } + bool withDictionary() const override { return true; } + private: ColumnPtr column_unique; ColumnPtr indexes; diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 40577a11d3f..2b32348204c 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -312,6 +312,8 @@ public: /// Can be inside ColumnNullable. virtual bool canBeInsideNullable() const { return false; } + virtual bool withDictionary() const { return false; } + virtual ~IColumn() {} diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index e7820b9f0a7..bd2e0c2ca64 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -319,6 +319,7 @@ public: bool isEnum() const override { return false; } bool isNullable() const override { return false; } bool onlyNull() const override { return false; } + bool withDictionary() const override { return true; } }; } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 943e9c9e69f..e6542c18b70 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -244,71 +244,71 @@ public: /** Can appear in table definition. * Counterexamples: Interval, Nothing. */ - virtual bool cannotBeStoredInTables() const { return false; }; + virtual bool cannotBeStoredInTables() const { return false; } /** In text formats that render "pretty" tables, * is it better to align value right in table cell. * Examples: numbers, even nullable. */ - virtual bool shouldAlignRightInPrettyFormats() const { return false; }; + virtual bool shouldAlignRightInPrettyFormats() const { return false; } /** Does formatted value in any text format can contain anything but valid UTF8 sequences. * Example: String (because it can contain arbitary bytes). * Counterexamples: numbers, Date, DateTime. * For Enum, it depends. */ - virtual bool textCanContainOnlyValidUTF8() const { return false; }; + virtual bool textCanContainOnlyValidUTF8() const { return false; } /** Is it possible to compare for less/greater, to calculate min/max? * Not necessarily totally comparable. For example, floats are comparable despite the fact that NaNs compares to nothing. * The same for nullable of comparable types: they are comparable (but not totally-comparable). */ - virtual bool isComparable() const { return false; }; + virtual bool isComparable() const { return false; } /** Does it make sense to use this type with COLLATE modifier in ORDER BY. * Example: String, but not FixedString. */ - virtual bool canBeComparedWithCollation() const { return false; }; + virtual bool canBeComparedWithCollation() const { return false; } /** If the type is totally comparable (Ints, Date, DateTime, not nullable, not floats) * and "simple" enough (not String, FixedString) to be used as version number * (to select rows with maximum version). */ - virtual bool canBeUsedAsVersion() const { return false; }; + virtual bool canBeUsedAsVersion() const { return false; } /** Values of data type can be summed (possibly with overflow, within the same data type). * Example: numbers, even nullable. Not Date/DateTime. Not Enum. * Enums can be passed to aggregate function 'sum', but the result is Int64, not Enum, so they are not summable. */ - virtual bool isSummable() const { return false; }; + virtual bool isSummable() const { return false; } /** Can be used in operations like bit and, bit shift, bit not, etc. */ - virtual bool canBeUsedInBitOperations() const { return false; }; + virtual bool canBeUsedInBitOperations() const { return false; } /** Can be used in boolean context (WHERE, HAVING). * UInt8, maybe nullable. */ - virtual bool canBeUsedInBooleanContext() const { return false; }; + virtual bool canBeUsedInBooleanContext() const { return false; } /** Integers, floats, not Nullable. Not Enums. Not Date/DateTime. */ - virtual bool isNumber() const { return false; }; + virtual bool isNumber() const { return false; } /** Integers. Not Nullable. Not Enums. Not Date/DateTime. */ - virtual bool isInteger() const { return false; }; - virtual bool isUnsignedInteger() const { return false; }; + virtual bool isInteger() const { return false; } + virtual bool isUnsignedInteger() const { return false; } - virtual bool isDateOrDateTime() const { return false; }; + virtual bool isDateOrDateTime() const { return false; } /** Numbers, Enums, Date, DateTime. Not nullable. */ - virtual bool isValueRepresentedByNumber() const { return false; }; + virtual bool isValueRepresentedByNumber() const { return false; } /** Integers, Enums, Date, DateTime. Not nullable. */ - virtual bool isValueRepresentedByInteger() const { return false; }; + virtual bool isValueRepresentedByInteger() const { return false; } /** Values are unambiguously identified by contents of contiguous memory region, * that can be obtained by IColumn::getDataAt method. @@ -317,7 +317,7 @@ public: * (because Array(String) values became ambiguous if you concatenate Strings). * Counterexamples: Nullable, Tuple. */ - virtual bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const { return false; }; + virtual bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const { return false; } virtual bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const { @@ -326,13 +326,13 @@ public: virtual bool isString() const { return false; }; virtual bool isFixedString() const { return false; }; - virtual bool isStringOrFixedString() const { return isString() || isFixedString(); }; + virtual bool isStringOrFixedString() const { return isString() || isFixedString(); } /** Example: numbers, Date, DateTime, FixedString, Enum... Nullable and Tuple of such types. * Counterexamples: String, Array. * It's Ok to return false for AggregateFunction despite the fact that some of them have fixed size state. */ - virtual bool haveMaximumSizeOfValue() const { return false; }; + virtual bool haveMaximumSizeOfValue() const { return false; } /** Size in amount of bytes in memory. Throws an exception if not haveMaximumSizeOfValue. */ @@ -358,6 +358,8 @@ public: */ virtual bool canBeInsideNullable() const { return false; }; + virtual bool withDictionary() const { return false; } + /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 12e8dfabbd8..e6bdc48c5e5 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -8,6 +8,8 @@ #include #include #include +#include +#include namespace DB @@ -176,7 +178,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo for (size_t i = 0; i < arguments_size; ++i) temporary_argument_numbers[i] = i; - execute(temporary_block, temporary_argument_numbers, arguments_size); + executeWithoutColumnsWithDictionary(temporary_block, temporary_argument_numbers, arguments_size); block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column, block.rows()); return true; @@ -199,7 +201,7 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co if (null_presence.has_nullable) { Block temporary_block = createBlockWithNestedColumns(block, args, result); - execute(temporary_block, args, result); + executeWithoutColumnsWithDictionary(temporary_block, args, result); block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args, result); return true; } @@ -207,7 +209,8 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co return false; } -void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result) + +void PreparedFunctionImpl::executeWithoutColumnsWithDictionary(Block & block, const ColumnNumbers & args, size_t result) { if (defaultImplementationForConstantArguments(block, args, result)) return; @@ -218,6 +221,78 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si executeImpl(block, args, result); } +static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & args, size_t result) +{ + bool has_with_dictionary = false; + bool convert_all_to_full = false; + size_t column_with_dict_size = 0; + + for (auto & arg : args) + { + if (auto * column_with_dict = typeid_cast(block.getByPosition(arg).column.get())) + { + if (has_with_dictionary) + convert_all_to_full = true; + else + { + has_with_dictionary = true; + column_with_dict_size = column_with_dict->getUnique()->size(); + } + } + } + + if (!has_with_dictionary) + return {}; + + Block temp_block; + temp_block.insert(block.getByPosition(result)); + { + auto & column = temp_block.getByPosition(0); + auto * type_with_dict = typeid_cast(column.type.get()); + if (!type_with_dict) + throw Exception("Return type of function which has argument WithDictionary must be WithDictionary, got" + + column.type->getName(), ErrorCodes::LOGICAL_ERROR); + + column.type = type_with_dict->getDictionaryType(); + } + + for (auto & arg : args) + { + auto & column = block.getByPosition(arg); + if (auto * column_with_dict = typeid_cast(column.column.get)) + { + auto * type_with_dict = typeid_cast(column.type.get()); + if (!type_with_dict) + throw Exception("Column with dictionary must have type WithDictionary, but has" + + column.type->getName(), ErrorCodes::LOGICAL_ERROR); + + ColumnPtr new_column = convert_all_to_full ? column_with_dict->convertToFullColumn() + : column_with_dict->getUnique()->getNestedColumn(); + + temp_block.insert({new_column, type_with_dict->getDictionaryType(), column.name}); + } + else if (auto * column_const = typeid_cast(column.column.get())) + temp_block.insert({column_const->cloneResized(column_with_dict_size), column.type, column.name}); + else if (convert_all_to_full) + temp_block.insert(column); + else + throw Exception("Expected ColumnWithDictionary or ColumnConst, got" + column.column->getName(), + ErrorCodes::LOGICAL_ERROR); + } + + return temp_block; +} + +void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result) +{ + if (useDefaultImplementationForColumnsWithDictionary()) + { + + } + + return executeWithoutColumnsWithDictionary(block, args, result); +} + void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const { if (isVariadic()) @@ -231,7 +306,36 @@ void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) con ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } -DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & arguments) const +struct ArgumentsWithoutDictionary +{ + ColumnsWithTypeAndName arguments; + DataTypePtr common_index_type; + bool all_without_dictionary = true; + + explicit ArgumentsWithoutDictionary(const ColumnsWithTypeAndName & args) + { + DataTypes index_types; + for (size_t i = 0; i < args.size(); ++i) + { + const auto & arg = args[i]; + if (auto * arg_with_dict = typeid_cast(arg.type.get())) + { + if (all_without_dictionary) + { + all_without_dictionary = false; + arguments = args; + } + arguments[i].type = arg_with_dict->getDictionaryType(); + index_types.push_back(arg_with_dict->getIndexesType()); + } + } + + if (!all_without_dictionary) + common_index_type = getLeastSupertype(index_types); + } +}; + +DataTypePtr FunctionBuilderImpl::getReturnTypeWithoutDictionary(const ColumnsWithTypeAndName & arguments) const { checkNumberOfArguments(arguments.size()); @@ -254,4 +358,18 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar return getReturnTypeImpl(arguments); } + +DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & arguments) const +{ + if (useDefaultImplementationForColumnsWithDictionary()) + { + ArgumentsWithoutDictionary arguments_without_dictionary(arguments); + if (!arguments_without_dictionary.all_without_dictionary) + return std::make_shared( + getReturnTypeWithoutDictionary(arguments_without_dictionary.arguments), + arguments_without_dictionary.common_index_type); + } + + return getReturnTypeWithoutDictionary(arguments); +} } diff --git a/dbms/src/Functions/IFunction.h b/dbms/src/Functions/IFunction.h index b7791268c79..21ad8b491d3 100644 --- a/dbms/src/Functions/IFunction.h +++ b/dbms/src/Functions/IFunction.h @@ -59,6 +59,12 @@ protected: */ virtual bool useDefaultImplementationForConstants() const { return false; } + /** If function arguments has single column with dictionary and all other arguments are constants, call function on nested column. + * Otherwise, convert all columns with dictionary to ordinary columns. + * Returns ColumnWithDictionary if at least one argument is ColumnWithDictionary. + */ + virtual bool useDefaultImplementationForColumnsWithDictionary() const { return true; } + /** Some arguments could remain constant during this implementation. */ virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; } @@ -66,6 +72,7 @@ protected: private: bool defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result); bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result); + void executeWithoutColumnsWithDictionary(Block & block, const ColumnNumbers & arguments, size_t result); }; /// Function with known arguments and return type. @@ -234,12 +241,22 @@ protected: */ virtual bool useDefaultImplementationForNulls() const { return true; } + /** If useDefaultImplementationForNulls() is true, than change arguments for getReturnType() and buildImpl(). + * If function arguments has types with dictionary, convert them to ordinary types. + * getReturnType returns ColumnWithDictionary if at least one argument type is ColumnWithDictionary. + */ + virtual bool useDefaultImplementationForColumnsWithDictionary() const { return true; } + virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const = 0; virtual void getLambdaArgumentTypesImpl(DataTypes & /*arguments*/) const { throw Exception("Function " + getName() + " can't have lambda-expressions as arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } + +private: + + DataTypePtr getReturnTypeWithoutDictionary(const ColumnsWithTypeAndName & arguments) const; }; /// Previous function interface. From 2119469c5166ce0fa3f2e1edd7d8d23019d233b3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 19 Apr 2018 00:00:47 +0300 Subject: [PATCH 012/315] added IColumn::select --- dbms/src/Columns/ColumnAggregateFunction.cpp | 20 ++++++++++ dbms/src/Columns/ColumnAggregateFunction.h | 5 +++ dbms/src/Columns/ColumnArray.cpp | 40 ++++++++++++++++++++ dbms/src/Columns/ColumnArray.h | 2 + dbms/src/Columns/ColumnConst.cpp | 12 ++++++ dbms/src/Columns/ColumnConst.h | 1 + dbms/src/Columns/ColumnVector.cpp | 36 ++++++++++++++++++ dbms/src/Columns/ColumnVector.h | 5 +++ dbms/src/Columns/ColumnsCommon.cpp | 20 ++++++++++ dbms/src/Columns/ColumnsCommon.h | 40 ++++++++++++++++++++ dbms/src/Columns/IColumn.h | 4 ++ 11 files changed, 185 insertions(+) diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index b989c007e56..b04a8fbd68f 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -161,6 +162,25 @@ ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limi return std::move(res); } +ColumnPtr ColumnAggregateFunction::index(const ColumnPtr & indexes, size_t limit) const +{ + return selectIndexImpl(*this, indexes, limit); +} + +template +ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + auto res = createView(); + + res->getData().resize(limit); + for (size_t i = 0; i < limit; ++i) + res->getData()[i] = getData()[indexes[i]]; + + return std::move(res); +} + +INSTANTIATE_INDEX_IMPL(ColumnAggregateFunction); + /// Is required to support operations with Set void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) const { diff --git a/dbms/src/Columns/ColumnAggregateFunction.h b/dbms/src/Columns/ColumnAggregateFunction.h index 852d76130d5..c0b64b69cda 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.h +++ b/dbms/src/Columns/ColumnAggregateFunction.h @@ -156,6 +156,11 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + + template + ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; + ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 010569e40ad..20250e4ba0b 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -626,6 +626,46 @@ ColumnPtr ColumnArray::permute(const Permutation & perm, size_t limit) const return std::move(res); } +ColumnPtr ColumnArray::index(const ColumnPtr & indexes, size_t limit) const +{ + return selectIndexImpl(*this, indexes, limit); +} + +template +ColumnPtr ColumnArray::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + size_t size = indexes.size(); + + if (limit == 0) + return ColumnArray::create(data); + + /// Convert indexes to UInt64 in case of overflow. + auto nested_indexes_column = ColumnUInt64::create(); + PaddedPODArray & nested_indexes = nested_indexes_column->getData(); + nested_indexes.reserve(getOffsets().back()); + + auto res = ColumnArray::create(data->cloneEmpty()); + + Offsets & res_offsets = res->getOffsets(); + res_offsets.resize(limit); + size_t current_offset = 0; + + for (size_t i = 0; i < limit; ++i) + { + for (size_t j = 0; j < sizeAt(indexes[i]); ++j) + nested_indexes.push_back(offsetAt(indexes[i]) + j); + current_offset += sizeAt(indexes[i]); + res_offsets[i] = current_offset; + } + + if (current_offset != 0) + res->data = data->index(nested_indexes_column, current_offset); + + return std::move(res); +} + +INSTANTIATE_INDEX_IMPL(ColumnArray); + void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { size_t s = size(); diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index ce7744b1fc1..1885a396a15 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -71,6 +71,8 @@ public: void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void reserve(size_t n) override; diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index 325160e5fc5..a0f7bc43567 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -63,6 +63,18 @@ ColumnPtr ColumnConst::permute(const Permutation & perm, size_t limit) const return ColumnConst::create(data, limit); } +ColumnPtr ColumnConst::index(const ColumnPtr & indexes, size_t limit) const +{ + if (limit == 0) + limit = indexes->size(); + + if (indexes->size() < limit) + throw Exception("Size of indexes (" + toString(indexes->size()) + ") is less than required (" + toString(limit) + ")", + ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + return ColumnConst::create(data, limit); +} + MutableColumns ColumnConst::scatter(ColumnIndex num_columns, const Selector & selector) const { if (s != selector.size()) diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index 2e4a692451f..c5de7a99c0f 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -148,6 +148,7 @@ public: ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr replicate(const Offsets & offsets) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; size_t byteSize() const override diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 3509c14c3aa..1552cd81e06 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -17,6 +17,8 @@ #if __SSE2__ #include +#include + #endif @@ -230,6 +232,40 @@ ColumnPtr ColumnVector::permute(const IColumn::Permutation & perm, size_t lim return std::move(res); } +template +ColumnPtr ColumnVector::index(const ColumnPtr & indexes, size_t limit) const +{ + return selectIndexImpl(*this, indexes, limit); +} + +template +template +ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + size_t size = indexes.size(); + + if (limit == 0) + limit = size; + else + limit = std::min(size, limit); + + auto res = this->create(limit); + typename Self::Container & res_data = res->getData(); + for (size_t i = 0; i < limit; ++i) + res_data[i] = data[indexes[i]]; + + return std::move(res); +} + +template +ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +template +ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +template +ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +template +ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; + template ColumnPtr ColumnVector::replicate(const IColumn::Offsets & offsets) const { diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 5ce33e82028..7175871042b 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -247,6 +247,11 @@ public: ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override; + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + + template + ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; + ColumnPtr replicate(const IColumn::Offsets & offsets) const override; void getExtremes(Field & min, Field & max) const override; diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index d4763697fe5..b068f18928e 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -3,6 +3,8 @@ #endif #include +#include +#include namespace DB @@ -308,4 +310,22 @@ INSTANTIATE(Float64) #undef INSTANTIATE +namespace detail +{ + template + PaddedPODArray * const getIndexesData(const ColumnPtr & indexes) + { + auto * column = typeid_cast *>(indexes.get()); + if (column) + return &column->getData(); + + return nullptr; + } +} + +PaddedPODArray * const detail::getIndexesData(const DB::ColumnPtr & indexes); +PaddedPODArray * const detail::getIndexesData(const DB::ColumnPtr & indexes); +PaddedPODArray * const detail::getIndexesData(const DB::ColumnPtr & indexes); +PaddedPODArray * const detail::getIndexesData(const DB::ColumnPtr & indexes); + } diff --git a/dbms/src/Columns/ColumnsCommon.h b/dbms/src/Columns/ColumnsCommon.h index a0048dee6c2..2373f00d8cf 100644 --- a/dbms/src/Columns/ColumnsCommon.h +++ b/dbms/src/Columns/ColumnsCommon.h @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + /// Counts how many bytes of `filt` are greater than zero. size_t countBytesInFilter(const IColumn::Filter & filt); @@ -33,4 +38,39 @@ void filterArraysImplOnlyData( PaddedPODArray & res_elems, const IColumn::Filter & filt, ssize_t result_size_hint); +namespace detail +{ + template + PaddedPODArray * const getIndexesData(const ColumnPtr & indexes); +} + +/// Check limit <= indexes->size() and call column.indexImpl(const PaddedPodArray & indexes, size_t limit). +template +ColumnPtr selectIndexImpl(const Column & column, const ColumnPtr & indexes, size_t limit) +{ + if (limit == 0) + limit = indexes->size(); + + if (indexes->size() < limit) + throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + if (auto * data_uint8 = detail::getIndexesData(indexes)) + return column.selectImpl(*data_uint8, limit); + else if (auto * data_uint16 = detail::getIndexesData(indexes)) + return column.selectImpl(*data_uint16, limit); + else if (auto * data_uint32 = detail::getIndexesData(indexes)) + return column.selectImpl(*data_uint32, limit); + else if (auto * data_uint64 = detail::getIndexesData(indexes)) + return column.selectImpl(*data_uint64, limit); + else + throw Exception("Indexes column for IColumn::select must be ColumnUInt, got" + indexes->getName(), + ErrorCodes::LOGICAL_ERROR); +} + +#define INSTANTIATE_INDEX_IMPL(Column) \ + template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ + template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ + template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ + template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; + } diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 2b32348204c..db59aa6e281 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -178,6 +178,10 @@ public: using Permutation = PaddedPODArray; virtual Ptr permute(const Permutation & perm, size_t limit) const = 0; + /// Creates new column with values column[indexes[:limit]]. If limit is 0, all indexes are used. + /// Indexes must be one of the ColumnUInt. For default implementation, see selectIndexImpl from ColumnsCommon.h + virtual Ptr index(const Ptr & indexes, size_t limit) const = 0; + /** Compares (*this)[n] and rhs[m]. * Returns negative number, 0, or positive number (*this)[n] is less, equal, greater than rhs[m] respectively. * Is used in sortings. From 4369613435346e3b850da311e3422618a0be09f4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 19:40:25 +0300 Subject: [PATCH 013/315] added IColumn::select --- dbms/src/Columns/ColumnFixedString.cpp | 27 ++++++++++++++++ dbms/src/Columns/ColumnFixedString.h | 5 +++ dbms/src/Columns/ColumnFunction.cpp | 9 ++++++ dbms/src/Columns/ColumnFunction.h | 1 + dbms/src/Columns/ColumnNullable.cpp | 7 +++++ dbms/src/Columns/ColumnNullable.h | 1 + dbms/src/Columns/ColumnString.cpp | 42 +++++++++++++++++++++++++ dbms/src/Columns/ColumnString.h | 5 +++ dbms/src/Columns/ColumnTuple.cpp | 11 +++++++ dbms/src/Columns/ColumnTuple.h | 1 + dbms/src/Columns/ColumnWithDictionary.h | 19 ++++------- dbms/src/Columns/IColumnDummy.h | 8 +++++ dbms/src/Columns/IColumnUnique.h | 5 +++ dbms/src/Functions/IFunction.cpp | 14 ++++++++- 14 files changed, 141 insertions(+), 14 deletions(-) diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index 2da3c28e50a..93f276a50ba 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -258,6 +259,32 @@ ColumnPtr ColumnFixedString::permute(const Permutation & perm, size_t limit) con return std::move(res); } + +ColumnPtr ColumnFixedString::index(const ColumnPtr & indexes, size_t limit) const +{ + return selectIndexImpl(*this, indexes, limit); +} + + +template +ColumnPtr ColumnFixedString::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + if (limit == 0) + return ColumnFixedString::create(n); + + auto res = ColumnFixedString::create(n); + + Chars_t & res_chars = res->chars; + + res_chars.resize(n * limit); + + size_t offset = 0; + for (size_t i = 0; i < limit; ++i, offset += n) + memcpySmallAllowReadWriteOverflow15(&res_chars[offset], &chars[indexes[i] * n], n); + + return std::move(res); +} + ColumnPtr ColumnFixedString::replicate(const Offsets & offsets) const { size_t col_size = size(); diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index cd465a1814d..a7eff637401 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -108,6 +108,11 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + + template + ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; + ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override diff --git a/dbms/src/Columns/ColumnFunction.cpp b/dbms/src/Columns/ColumnFunction.cpp index 4675e7915f7..338d7fb95ec 100644 --- a/dbms/src/Columns/ColumnFunction.cpp +++ b/dbms/src/Columns/ColumnFunction.cpp @@ -86,6 +86,15 @@ ColumnPtr ColumnFunction::permute(const Permutation & perm, size_t limit) const return ColumnFunction::create(limit, function, capture); } +ColumnPtr ColumnFunction::index(const ColumnPtr & indexes, size_t limit) const +{ + ColumnsWithTypeAndName capture = captured_columns; + for (auto & column : capture) + column.column = column.column->index(indexes, limit); + + return ColumnFunction::create(limit, function, capture); +} + std::vector ColumnFunction::scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const { diff --git a/dbms/src/Columns/ColumnFunction.h b/dbms/src/Columns/ColumnFunction.h index d10459175cc..24ef11df07f 100644 --- a/dbms/src/Columns/ColumnFunction.h +++ b/dbms/src/Columns/ColumnFunction.h @@ -33,6 +33,7 @@ public: ColumnPtr replicate(const Offsets & offsets) const override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; void insertDefault() override; void popBack(size_t n) override; std::vector scatter(IColumn::ColumnIndex num_columns, diff --git a/dbms/src/Columns/ColumnNullable.cpp b/dbms/src/Columns/ColumnNullable.cpp index 557a57b3c6d..5f1c5f5bab7 100644 --- a/dbms/src/Columns/ColumnNullable.cpp +++ b/dbms/src/Columns/ColumnNullable.cpp @@ -166,6 +166,13 @@ ColumnPtr ColumnNullable::permute(const Permutation & perm, size_t limit) const return ColumnNullable::create(permuted_data, permuted_null_map); } +ColumnPtr ColumnNullable::index(const ColumnPtr & indexes, size_t limit) const +{ + ColumnPtr indexed_data = getNestedColumn().index(indexes, limit); + ColumnPtr indexed_null_map = getNullMapColumn().index(indexes, limit); + return ColumnNullable::create(indexed_data, indexed_null_map); +} + int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const { /// NULL values share the properties of NaN values. diff --git a/dbms/src/Columns/ColumnNullable.h b/dbms/src/Columns/ColumnNullable.h index 4ac0f87b8da..f05adcc50e2 100644 --- a/dbms/src/Columns/ColumnNullable.h +++ b/dbms/src/Columns/ColumnNullable.h @@ -64,6 +64,7 @@ public: void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void reserve(size_t n) override; diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index ecb3bc6f62e..93bc7d944a0 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -162,6 +162,48 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const } +ColumnPtr ColumnString::index(const ColumnPtr & indexes, size_t limit) const +{ + return selectIndexImpl(*this, indexes, limit); +} + +template +ColumnPtr ColumnString::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + if (limit == 0) + return ColumnString::create(); + + auto res = ColumnString::create(); + + Chars_t & res_chars = res->chars; + Offsets & res_offsets = res->offsets; + + + size_t new_chars_size = 0; + for (size_t i = 0; i < limit; ++i) + new_chars_size += sizeAt(indexes[i]); + res_chars.resize(new_chars_size); + + res_offsets.resize(limit); + + Offset current_new_offset = 0; + + for (size_t i = 0; i < limit; ++i) + { + size_t j = indexes[i]; + size_t string_offset = j == 0 ? 0 : offsets[j - 1]; + size_t string_size = offsets[j] - string_offset; + + memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size); + + current_new_offset += string_size; + res_offsets[i] = current_new_offset; + } + + return std::move(res); +} + + template struct ColumnString::less { diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 0eda083f22e..d29d8abc0e3 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -220,6 +220,11 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + + template + ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; + void insertDefault() override { chars.push_back(0); diff --git a/dbms/src/Columns/ColumnTuple.cpp b/dbms/src/Columns/ColumnTuple.cpp index 558b1685276..4fa66ba6b1e 100644 --- a/dbms/src/Columns/ColumnTuple.cpp +++ b/dbms/src/Columns/ColumnTuple.cpp @@ -179,6 +179,17 @@ ColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) const return ColumnTuple::create(new_columns); } +ColumnPtr ColumnTuple::index(const ColumnPtr & indexes, size_t limit) const +{ + const size_t tuple_size = columns.size(); + Columns new_columns(tuple_size); + + for (size_t i = 0; i < tuple_size; ++i) + new_columns[i] = columns[i]->index(indexes, limit); + + return ColumnTuple::create(new_columns); +} + ColumnPtr ColumnTuple::replicate(const Offsets & offsets) const { const size_t tuple_size = columns.size(); diff --git a/dbms/src/Columns/ColumnTuple.h b/dbms/src/Columns/ColumnTuple.h index a91cb210212..648914219f9 100644 --- a/dbms/src/Columns/ColumnTuple.h +++ b/dbms/src/Columns/ColumnTuple.h @@ -60,6 +60,7 @@ public: void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; void gather(ColumnGathererStream & gatherer_stream) override; diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index c9b2fffce1c..e38010e22a5 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -39,19 +39,7 @@ public: ColumnPtr convertToFullColumn() const { - auto & nested = getUnique()->getNestedColumn(); - auto * indexes = getIndexes(); - if (auto * column_uint8 = typeid_cast(indexes)) - return nested->sample(column_uint8->getData()); - else if (auto * column_uint16 = typeid_cast(indexes)) - return nested->sample(column_uint16->getData()); - else if (auto * column_uint32 = typeid_cast(indexes)) - return nested->sample(column_uint32->getData()); - else if (auto * column_uint64 = typeid_cast(indexes)) - return nested->sample(column_uint64->getData()); - else - throw Exception("Indexes column for ColumnWithDictionary expected to be ColumnUInt, got " - + indexes->getName(), ErrorCodes::LOGICAL_ERROR); + return getUnique()->getNestedColumn()->index(indexes, 0); } MutableColumnPtr cloneResized(size_t size) const override @@ -135,6 +123,11 @@ public: return ColumnWithDictionary::create(column_unique, indexes->permute(perm, limit)); } + ColumnPtr index(const ColumnPtr & indexes_, size_t limit) const override + { + return ColumnWithDictionary::create(column_unique, indexes->index(indexes_, limit)); + } + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override { const auto & column_with_dictionary = static_cast(rhs); diff --git a/dbms/src/Columns/IColumnDummy.h b/dbms/src/Columns/IColumnDummy.h index c580a2b0cb1..7837d2b200e 100644 --- a/dbms/src/Columns/IColumnDummy.h +++ b/dbms/src/Columns/IColumnDummy.h @@ -87,6 +87,14 @@ public: return cloneDummy(limit ? std::min(s, limit) : s); } + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override + { + if (indexes->size() < limit) + throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + return cloneDummy(limit ? limit : s); + } + void getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const override { res.resize(s); diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index e136c12c697..956fe78ec76 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -74,6 +74,11 @@ public: { throw Exception("Method deserializeAndInsertFromArena is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } + + ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override + { + throw Exception("Method index is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } }; } diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index e6bdc48c5e5..31a5472b124 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -287,10 +287,22 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si { if (useDefaultImplementationForColumnsWithDictionary()) { + Block temp_block = removeColumnsWithDictionary(block, args, result); + if (temp_block) + { + ColumnNumbers temp_numbers(args.size()); + for (size_t i = 0; i < args.size(); ++i) + temp_numbers[i] = i + 1; + executeWithoutColumnsWithDictionary(temp_block, temp_numbers, 0); + auto & res_col = block.getByPosition(result); + res_col.column = res_col.type->createColumn(); + res_col.column->insertRangeFrom(*temp_block.getByPosition(0).column, 0, temp_block.rows()); + return; + } } - return executeWithoutColumnsWithDictionary(block, args, result); + executeWithoutColumnsWithDictionary(block, args, result); } void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const From 44807df50fbba6843312ae5e6b722a6c4fa587f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 19:48:46 +0300 Subject: [PATCH 014/315] added IColumn::select --- dbms/src/Columns/ColumnsCommon.cpp | 10 +++++----- dbms/src/Columns/ColumnsCommon.h | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index b068f18928e..f8a3a3503e2 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -313,7 +313,7 @@ INSTANTIATE(Float64) namespace detail { template - PaddedPODArray * const getIndexesData(const ColumnPtr & indexes) + const PaddedPODArray * getIndexesData(const ColumnPtr & indexes) { auto * column = typeid_cast *>(indexes.get()); if (column) @@ -323,9 +323,9 @@ namespace detail } } -PaddedPODArray * const detail::getIndexesData(const DB::ColumnPtr & indexes); -PaddedPODArray * const detail::getIndexesData(const DB::ColumnPtr & indexes); -PaddedPODArray * const detail::getIndexesData(const DB::ColumnPtr & indexes); -PaddedPODArray * const detail::getIndexesData(const DB::ColumnPtr & indexes); +const PaddedPODArray * detail::getIndexesData(const DB::ColumnPtr & indexes); +const PaddedPODArray * detail::getIndexesData(const DB::ColumnPtr & indexes); +const PaddedPODArray * detail::getIndexesData(const DB::ColumnPtr & indexes); +const PaddedPODArray * detail::getIndexesData(const DB::ColumnPtr & indexes); } diff --git a/dbms/src/Columns/ColumnsCommon.h b/dbms/src/Columns/ColumnsCommon.h index 2373f00d8cf..4528462afe4 100644 --- a/dbms/src/Columns/ColumnsCommon.h +++ b/dbms/src/Columns/ColumnsCommon.h @@ -41,7 +41,7 @@ void filterArraysImplOnlyData( namespace detail { template - PaddedPODArray * const getIndexesData(const ColumnPtr & indexes); + const PaddedPODArray * getIndexesData(const ColumnPtr & indexes); } /// Check limit <= indexes->size() and call column.indexImpl(const PaddedPodArray & indexes, size_t limit). @@ -55,13 +55,13 @@ ColumnPtr selectIndexImpl(const Column & column, const ColumnPtr & indexes, size throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); if (auto * data_uint8 = detail::getIndexesData(indexes)) - return column.selectImpl(*data_uint8, limit); + return column.template selectImpl(*data_uint8, limit); else if (auto * data_uint16 = detail::getIndexesData(indexes)) - return column.selectImpl(*data_uint16, limit); + return column.template selectImpl(*data_uint16, limit); else if (auto * data_uint32 = detail::getIndexesData(indexes)) - return column.selectImpl(*data_uint32, limit); + return column.template selectImpl(*data_uint32, limit); else if (auto * data_uint64 = detail::getIndexesData(indexes)) - return column.selectImpl(*data_uint64, limit); + return column.template selectImpl(*data_uint64, limit); else throw Exception("Indexes column for IColumn::select must be ColumnUInt, got" + indexes->getName(), ErrorCodes::LOGICAL_ERROR); From 61e14b8a8368844295a01339dd0254a2c3098125 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 19:51:27 +0300 Subject: [PATCH 015/315] added IColumn::select --- dbms/src/Columns/ColumnsCommon.h | 8 ++++---- dbms/src/Columns/IColumnUnique.h | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Columns/ColumnsCommon.h b/dbms/src/Columns/ColumnsCommon.h index 4528462afe4..8ef90f48625 100644 --- a/dbms/src/Columns/ColumnsCommon.h +++ b/dbms/src/Columns/ColumnsCommon.h @@ -55,13 +55,13 @@ ColumnPtr selectIndexImpl(const Column & column, const ColumnPtr & indexes, size throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); if (auto * data_uint8 = detail::getIndexesData(indexes)) - return column.template selectImpl(*data_uint8, limit); + return column.template indexImpl(*data_uint8, limit); else if (auto * data_uint16 = detail::getIndexesData(indexes)) - return column.template selectImpl(*data_uint16, limit); + return column.template indexImpl(*data_uint16, limit); else if (auto * data_uint32 = detail::getIndexesData(indexes)) - return column.template selectImpl(*data_uint32, limit); + return column.template indexImpl(*data_uint32, limit); else if (auto * data_uint64 = detail::getIndexesData(indexes)) - return column.template selectImpl(*data_uint64, limit); + return column.template indexImpl(*data_uint64, limit); else throw Exception("Indexes column for IColumn::select must be ColumnUInt, got" + indexes->getName(), ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index 956fe78ec76..36ba42d1889 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -75,7 +75,7 @@ public: throw Exception("Method deserializeAndInsertFromArena is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override + ColumnPtr index(const ColumnPtr &, size_t) const override { throw Exception("Method index is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } From 4aaf56d9a2a6fab8755e1dc557cb8bc8a4a3f078 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 19:55:26 +0300 Subject: [PATCH 016/315] added IColumn::select --- dbms/src/Columns/ColumnArray.cpp | 2 -- dbms/src/Columns/ColumnsCommon.cpp | 10 +++++----- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 20250e4ba0b..118bfd8b517 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -634,8 +634,6 @@ ColumnPtr ColumnArray::index(const ColumnPtr & indexes, size_t limit) const template ColumnPtr ColumnArray::indexImpl(const PaddedPODArray & indexes, size_t limit) const { - size_t size = indexes.size(); - if (limit == 0) return ColumnArray::create(data); diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index f8a3a3503e2..e2f45242ddc 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -321,11 +321,11 @@ namespace detail return nullptr; } + + const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); } -const PaddedPODArray * detail::getIndexesData(const DB::ColumnPtr & indexes); -const PaddedPODArray * detail::getIndexesData(const DB::ColumnPtr & indexes); -const PaddedPODArray * detail::getIndexesData(const DB::ColumnPtr & indexes); -const PaddedPODArray * detail::getIndexesData(const DB::ColumnPtr & indexes); - } From 8470a091bb36543d3b9ad80e4fa952f1c412226f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 19:59:11 +0300 Subject: [PATCH 017/315] added IColumn::select --- dbms/src/Columns/ColumnArray.cpp | 2 +- dbms/src/Columns/ColumnsCommon.cpp | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 118bfd8b517..085c21673f4 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -657,7 +657,7 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray & indexes, size_t limit } if (current_offset != 0) - res->data = data->index(nested_indexes_column, current_offset); + res->data = data->index(std::move(nested_indexes_column), current_offset); return std::move(res); } diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index e2f45242ddc..1fb24261007 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -322,9 +322,13 @@ namespace detail return nullptr; } + template <> const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + template <> const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + template <> const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + template <> const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); } From 387c32e1ec42fac490966b1bf0f4b394fa9b770a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 20:01:28 +0300 Subject: [PATCH 018/315] added IColumn::select --- dbms/src/Columns/ColumnVector.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 1552cd81e06..6279f3f1fc5 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -258,12 +258,16 @@ ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_ } template +template <> ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; template +template <> ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; template +template <> ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; template +template <> ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; template From e6e29eef295851e567bb6802b2826fcaf743421f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 20:02:50 +0300 Subject: [PATCH 019/315] added IColumn::select --- dbms/src/Columns/ColumnVector.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 6279f3f1fc5..0e8d7427410 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -257,17 +257,17 @@ ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_ return std::move(res); } -template template <> +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; -template template <> +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; -template template <> +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; -template template <> +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; template From 8625686205adeafd9ef674787574570e424df258 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 20:19:24 +0300 Subject: [PATCH 020/315] added IColumn::select --- dbms/src/Columns/ColumnVector.cpp | 32 ------------------------------- dbms/src/Columns/ColumnVector.h | 18 +++++++++++++++++ 2 files changed, 18 insertions(+), 32 deletions(-) diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 0e8d7427410..582e3a0afa5 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -238,38 +238,6 @@ ColumnPtr ColumnVector::index(const ColumnPtr & indexes, size_t limit) const return selectIndexImpl(*this, indexes, limit); } -template -template -ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const -{ - size_t size = indexes.size(); - - if (limit == 0) - limit = size; - else - limit = std::min(size, limit); - - auto res = this->create(limit); - typename Self::Container & res_data = res->getData(); - for (size_t i = 0; i < limit; ++i) - res_data[i] = data[indexes[i]]; - - return std::move(res); -} - -template <> -template -ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; -template <> -template -ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; -template <> -template -ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; -template <> -template -ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; - template ColumnPtr ColumnVector::replicate(const IColumn::Offsets & offsets) const { diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 7175871042b..16e80a2c42e 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -295,5 +295,23 @@ protected: Container data; }; +template +template +ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + size_t size = indexes.size(); + + if (limit == 0) + limit = size; + else + limit = std::min(size, limit); + + auto res = this->create(limit); + typename Self::Container & res_data = res->getData(); + for (size_t i = 0; i < limit; ++i) + res_data[i] = data[indexes[i]]; + + return std::move(res); +} } From 46d47cd892fb9e66d30362b1210e716b5806f2c0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 21:19:34 +0300 Subject: [PATCH 021/315] added IColumn::select --- dbms/src/Functions/IFunction.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 31a5472b124..15fc0939e71 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -259,7 +259,7 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar for (auto & arg : args) { auto & column = block.getByPosition(arg); - if (auto * column_with_dict = typeid_cast(column.column.get)) + if (auto * column_with_dict = typeid_cast(column.column.get())) { auto * type_with_dict = typeid_cast(column.type.get()); if (!type_with_dict) @@ -297,7 +297,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si executeWithoutColumnsWithDictionary(temp_block, temp_numbers, 0); auto & res_col = block.getByPosition(result); res_col.column = res_col.type->createColumn(); - res_col.column->insertRangeFrom(*temp_block.getByPosition(0).column, 0, temp_block.rows()); + res_col.column->insertRangeFrom(temp_block.getByPosition(0).column->assumeMutableRef(), 0, temp_block.rows()); return; } } From 792caf4294263ffaf09d9abf0e63718d7057f886 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Apr 2018 21:20:52 +0300 Subject: [PATCH 022/315] added IColumn::select --- dbms/src/Functions/IFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 15fc0939e71..4af29540af6 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -297,7 +297,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si executeWithoutColumnsWithDictionary(temp_block, temp_numbers, 0); auto & res_col = block.getByPosition(result); res_col.column = res_col.type->createColumn(); - res_col.column->insertRangeFrom(temp_block.getByPosition(0).column->assumeMutableRef(), 0, temp_block.rows()); + res_col.column->assumeMutableRef().insertRangeFrom(*temp_block.getByPosition(0).column, 0, temp_block.rows()); return; } } From b662dcdd0a032b744f8ba90c033ed16c18d6af45 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 13:53:00 +0300 Subject: [PATCH 023/315] added IColumn::select --- dbms/src/Columns/ColumnsCommon.cpp | 8 ++++---- dbms/src/Functions/FunctionsMiscellaneous.h | 2 ++ dbms/src/Functions/IFunction.h | 3 +++ 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index 1fb24261007..e07f4171cac 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -322,13 +322,13 @@ namespace detail return nullptr; } - template <> + template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - template <> + template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - template <> + template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - template <> + template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); } diff --git a/dbms/src/Functions/FunctionsMiscellaneous.h b/dbms/src/Functions/FunctionsMiscellaneous.h index 78184e81105..47d355d9d1a 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.h +++ b/dbms/src/Functions/FunctionsMiscellaneous.h @@ -78,6 +78,7 @@ public: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { @@ -110,6 +111,7 @@ public: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { diff --git a/dbms/src/Functions/IFunction.h b/dbms/src/Functions/IFunction.h index 21ad8b491d3..5aaeb81bdc5 100644 --- a/dbms/src/Functions/IFunction.h +++ b/dbms/src/Functions/IFunction.h @@ -271,6 +271,7 @@ public: /// Override this functions to change default implementation behavior. See details in IMyFunction. bool useDefaultImplementationForNulls() const override { return true; } bool useDefaultImplementationForConstants() const override { return false; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } using PreparedFunctionImpl::execute; @@ -317,6 +318,7 @@ protected: } bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); } bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); } + bool useDefaultImplementationForColumnsWithDictionary() const final { return function->useDefaultImplementationForColumnsWithDictionary(); } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); } private: @@ -375,6 +377,7 @@ protected: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); } bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); } + bool useDefaultImplementationForColumnsWithDictionary() const override { return function->useDefaultImplementationForColumnsWithDictionary(); } FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { From 882ddff7d4cd9b3de85c010d1102448713f2ebcc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 14:59:01 +0300 Subject: [PATCH 024/315] added IColumn::select --- dbms/src/Columns/ColumnsCommon.cpp | 12 ++---- dbms/src/Functions/FunctionsMiscellaneous.cpp | 1 + dbms/src/Functions/IFunction.cpp | 39 ++++++++++++++----- 3 files changed, 35 insertions(+), 17 deletions(-) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index e07f4171cac..db6abde09ac 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -322,14 +322,10 @@ namespace detail return nullptr; } - template - const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - template - const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - template - const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - template - const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); } } diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index a953adb9e8e..c15614d27a8 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -210,6 +210,7 @@ public: } bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } size_t getNumberOfArguments() const override { diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 4af29540af6..989cc6df82b 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -221,7 +221,7 @@ void PreparedFunctionImpl::executeWithoutColumnsWithDictionary(Block & block, co executeImpl(block, args, result); } -static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & args, size_t result) +static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & args, size_t result, ColumnPtr & indexes) { bool has_with_dictionary = false; bool convert_all_to_full = false; @@ -229,7 +229,8 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar for (auto & arg : args) { - if (auto * column_with_dict = typeid_cast(block.getByPosition(arg).column.get())) + const auto & column = block.getByPosition(arg).column; + if (auto * column_with_dict = checkAndGetColumn(column.get())) { if (has_with_dictionary) convert_all_to_full = true; @@ -237,10 +238,16 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar { has_with_dictionary = true; column_with_dict_size = column_with_dict->getUnique()->size(); + indexes = column_with_dict->getIndexesPtr(); } } + else if (!checkColumnConst(column.get())) + convert_all_to_full = true; } + if (!has_with_dictionary || convert_all_to_full) + indexes = nullptr; + if (!has_with_dictionary) return {}; @@ -248,7 +255,7 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar temp_block.insert(block.getByPosition(result)); { auto & column = temp_block.getByPosition(0); - auto * type_with_dict = typeid_cast(column.type.get()); + auto * type_with_dict = checkAndGetDataType(column.type.get()); if (!type_with_dict) throw Exception("Return type of function which has argument WithDictionary must be WithDictionary, got" + column.type->getName(), ErrorCodes::LOGICAL_ERROR); @@ -259,9 +266,9 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar for (auto & arg : args) { auto & column = block.getByPosition(arg); - if (auto * column_with_dict = typeid_cast(column.column.get())) + if (auto * column_with_dict = checkAndGetColumn(column.column.get())) { - auto * type_with_dict = typeid_cast(column.type.get()); + auto * type_with_dict = checkAndGetDataType(column.type.get()); if (!type_with_dict) throw Exception("Column with dictionary must have type WithDictionary, but has" + column.type->getName(), ErrorCodes::LOGICAL_ERROR); @@ -271,7 +278,7 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar temp_block.insert({new_column, type_with_dict->getDictionaryType(), column.name}); } - else if (auto * column_const = typeid_cast(column.column.get())) + else if (auto * column_const = checkAndGetColumnConst(column.column.get())) temp_block.insert({column_const->cloneResized(column_with_dict_size), column.type, column.name}); else if (convert_all_to_full) temp_block.insert(column); @@ -287,7 +294,8 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si { if (useDefaultImplementationForColumnsWithDictionary()) { - Block temp_block = removeColumnsWithDictionary(block, args, result); + ColumnPtr indexes; + Block temp_block = removeColumnsWithDictionary(block, args, result, indexes); if (temp_block) { ColumnNumbers temp_numbers(args.size()); @@ -295,9 +303,22 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si temp_numbers[i] = i + 1; executeWithoutColumnsWithDictionary(temp_block, temp_numbers, 0); + auto & temp_res_col = temp_block.getByPosition(0).column; auto & res_col = block.getByPosition(result); - res_col.column = res_col.type->createColumn(); - res_col.column->assumeMutableRef().insertRangeFrom(*temp_block.getByPosition(0).column, 0, temp_block.rows()); + if (indexes) + res_col.column = ColumnWithDictionary::create(ColumnUnique::create((*std::move(temp_res_col)).mutate(), + (*std::move(indexes)).mutate())); + else + { + res_col.column = res_col.type->createColumn(); + + auto * col_with_dict = checkAndGetColumn(res_col.column.get()); + if (!col_with_dict) + throw Exception("Expected ColumnWithDictionary, got" + res_col.column->getName(), + ErrorCodes::LOGICAL_ERROR); + + col_with_dict->assumeMutableRef().insertRangeFrom(*temp_res_col, 0, temp_res_col->size()); + } return; } } From 4ac8078c82114dee862d6a6c80feb14dc3b6154e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 15:02:36 +0300 Subject: [PATCH 025/315] added IColumn::select --- dbms/src/Functions/IFunction.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 989cc6df82b..30e7a8f369b 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -241,7 +241,7 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar indexes = column_with_dict->getIndexesPtr(); } } - else if (!checkColumnConst(column.get())) + else if (!checkColumn(column.get())) convert_all_to_full = true; } @@ -278,7 +278,7 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar temp_block.insert({new_column, type_with_dict->getDictionaryType(), column.name}); } - else if (auto * column_const = checkAndGetColumnConst(column.column.get())) + else if (auto * column_const = checkAndGetColumn(column.column.get())) temp_block.insert({column_const->cloneResized(column_with_dict_size), column.type, column.name}); else if (convert_all_to_full) temp_block.insert(column); From 97fbd37cb099cd244ee811f3d1b9b8b246011bed Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 15:47:14 +0300 Subject: [PATCH 026/315] added IColumn::select --- dbms/src/Columns/ColumnWithDictionary.h | 2 ++ dbms/src/Functions/IFunction.cpp | 24 ++++++++++++++---------- 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index e38010e22a5..f0bd2230480 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -212,6 +212,8 @@ public: const IColumn * getIndexes() const { return indexes.get(); } const ColumnPtr & getIndexesPtr() const { return indexes; } + void setIndexes(MutableColumnPtr && indexes_) { indexes = indexes_; } + bool withDictionary() const override { return true; } private: diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 30e7a8f369b..061147d3065 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -305,19 +305,23 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si executeWithoutColumnsWithDictionary(temp_block, temp_numbers, 0); auto & temp_res_col = temp_block.getByPosition(0).column; auto & res_col = block.getByPosition(result); + res_col.column = res_col.type->createColumn(); + + auto * col_with_dict = checkAndGetColumn(res_col.column.get()); + if (!col_with_dict) + throw Exception("Expected ColumnWithDictionary, got" + res_col.column->getName(), + ErrorCodes::LOGICAL_ERROR); + + ColumnWithDictionary & mut_col_with_dict = col_with_dict->assumeMutableRef(); + if (indexes) - res_col.column = ColumnWithDictionary::create(ColumnUnique::create((*std::move(temp_res_col)).mutate(), - (*std::move(indexes)).mutate())); + { + auto new_ind = mut_col_with_dict.getUnique()->uniqueInsertRangeFrom(*temp_res_col, 0, temp_res_col->size()); + mut_col_with_dict.setIndexes(new_ind->index(indexes, 0)->assumeMutable()); + } else { - res_col.column = res_col.type->createColumn(); - - auto * col_with_dict = checkAndGetColumn(res_col.column.get()); - if (!col_with_dict) - throw Exception("Expected ColumnWithDictionary, got" + res_col.column->getName(), - ErrorCodes::LOGICAL_ERROR); - - col_with_dict->assumeMutableRef().insertRangeFrom(*temp_res_col, 0, temp_res_col->size()); + mut_col_with_dict.insertRangeFrom(*temp_res_col, 0, temp_res_col->size()); } return; } From ba0a5af437a9fcc8eba38710d86f8f61cf12263b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 15:51:26 +0300 Subject: [PATCH 027/315] added IColumn::select --- dbms/src/Columns/ColumnWithDictionary.h | 2 +- dbms/src/Functions/IFunction.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index f0bd2230480..38fffebda58 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -212,7 +212,7 @@ public: const IColumn * getIndexes() const { return indexes.get(); } const ColumnPtr & getIndexesPtr() const { return indexes; } - void setIndexes(MutableColumnPtr && indexes_) { indexes = indexes_; } + void setIndexes(MutableColumnPtr && indexes_) { indexes = std::move(indexes_); } bool withDictionary() const override { return true; } diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 061147d3065..c0424b1b4e7 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -312,7 +312,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si throw Exception("Expected ColumnWithDictionary, got" + res_col.column->getName(), ErrorCodes::LOGICAL_ERROR); - ColumnWithDictionary & mut_col_with_dict = col_with_dict->assumeMutableRef(); + auto & mut_col_with_dict = static_cast(*col_with_dict); if (indexes) { From db23e0fcb4bc31e04e06b49300e1d469a4df74cb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 15:53:18 +0300 Subject: [PATCH 028/315] added IColumn::select --- dbms/src/Functions/IFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index c0424b1b4e7..810ce989b71 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -312,7 +312,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si throw Exception("Expected ColumnWithDictionary, got" + res_col.column->getName(), ErrorCodes::LOGICAL_ERROR); - auto & mut_col_with_dict = static_cast(*col_with_dict); + auto & mut_col_with_dict = const_cast(*col_with_dict); if (indexes) { From ce3b588de01775ae4f452521f0714fb5c43c98b8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 16:19:50 +0300 Subject: [PATCH 029/315] added IColumn::select --- dbms/src/DataTypes/DataTypeWithDictionary.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index bd2e0c2ca64..73fb4a4e48f 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -75,7 +75,7 @@ public: path.push_back(Substream::DictionaryElements); if (auto stream = getter(path)) - dictionary_type->serializeBinaryBulk(*column_with_dictionary.getUnique(), *stream, offset, limit); + dictionary_type->serializeBinaryBulk(*column_with_dictionary.getUnique()->getNestedColumn(), *stream, offset, limit); path.back() = Substream::DictionaryIndexes; if (auto stream = getter(path)) @@ -94,7 +94,7 @@ public: path.push_back(Substream::DictionaryElements); if (ReadBuffer * stream = getter(path)) - dictionary_type->deserializeBinaryBulk(*column_with_dictionary.getUnique(), *stream, limit, 0); + dictionary_type->deserializeBinaryBulk(*column_with_dictionary.getUnique()->getNestedColumn(), *stream, limit, 0); path.back() = Substream::DictionaryIndexes; if (auto stream = getter(path)) From 0806b957aace3377d7d2ffc0a288e20e3923881f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 16:45:11 +0300 Subject: [PATCH 030/315] added IColumn::select --- dbms/src/DataTypes/DataTypeWithDictionary.h | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index 73fb4a4e48f..a86c89486f6 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -75,7 +75,10 @@ public: path.push_back(Substream::DictionaryElements); if (auto stream = getter(path)) - dictionary_type->serializeBinaryBulk(*column_with_dictionary.getUnique()->getNestedColumn(), *stream, offset, limit); + { + if (offset == 0) + dictionary_type->serializeBinaryBulk(*column_with_dictionary.getUnique()->getNestedColumn(), *stream, 0, 0); + } path.back() = Substream::DictionaryIndexes; if (auto stream = getter(path)) @@ -94,7 +97,16 @@ public: path.push_back(Substream::DictionaryElements); if (ReadBuffer * stream = getter(path)) - dictionary_type->deserializeBinaryBulk(*column_with_dictionary.getUnique()->getNestedColumn(), *stream, limit, 0); + { + if (column.empty()) + { + auto dict_column = column_with_dictionary.getUnique()->getNestedColumn()->cloneEmpty(); + dictionary_type->deserializeBinaryBulk(*dict_column, *stream, 0, 0); + + /// Note: it's assumed that rows inserted into columnUnique get incremental indexes. + column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, dict_column->size()); + } + } path.back() = Substream::DictionaryIndexes; if (auto stream = getter(path)) From 98ff5b0a559f7242ef35ae4190c20d11b3858b29 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 17:00:24 +0300 Subject: [PATCH 031/315] Register DataTypeWithDictionary in DataTypeFactory --- dbms/src/DataTypes/DataTypeFactory.cpp | 2 ++ dbms/src/DataTypes/DataTypeWithDictionary.cpp | 29 +++++++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index f1a12d75868..ed6b5ad5d1f 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -118,6 +118,7 @@ void registerDataTypeUUID(DataTypeFactory & factory); void registerDataTypeAggregateFunction(DataTypeFactory & factory); void registerDataTypeNested(DataTypeFactory & factory); void registerDataTypeInterval(DataTypeFactory & factory); +void registerDataTypeWithDictionary(DataTypeFactory & factory); DataTypeFactory::DataTypeFactory() @@ -136,6 +137,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeAggregateFunction(*this); registerDataTypeNested(*this); registerDataTypeInterval(*this); + registerDataTypeWithDictionary(*this); } } diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index e69de29bb2d..ba19d9bbcf0 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -0,0 +1,29 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +static DataTypePtr create(const ASTPtr & arguments) +{ + if (!arguments || arguments->children.size() != 2) + throw Exception("WithDictionary data type family must have two arguments - type of elements and type of indices" + , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return std::make_shared(DataTypeFactory::instance().get(arguments->children[0]), + DataTypeFactory::instance().get(arguments->children[1])); +} + +void registerDataTypeWithDictionary(DataTypeFactory & factory) +{ + factory.registerDataType("WithDictionary", create); +} + +} From 04a056f804bfb1c7e4eebb653c19d1b66301d5a2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 17:53:35 +0300 Subject: [PATCH 032/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 213 +++++++++++++++ dbms/src/DataTypes/DataTypeWithDictionary.h | 251 ++++-------------- 2 files changed, 259 insertions(+), 205 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index ba19d9bbcf0..bcd0571ff5e 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -1,5 +1,13 @@ +#include +#include +#include +#include +#include #include #include +#include +#include +#include #include namespace DB @@ -8,8 +16,213 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_, DataTypePtr indexes_type_) + : dictionary_type(std::move(dictionary_type_)), indexes_type(std::move(indexes_type_)) +{ + if (!indexes_type->isUnsignedInteger()) + throw Exception("Index type of DataTypeWithDictionary must be unsigned integer, but got " + + indexes_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto inner_type = dictionary_type; + if (dictionary_type->isNullable()) + inner_type = static_cast(*dictionary_type).getNestedType(); + + if (!inner_type->isStringOrFixedString() + && !inner_type->isDateOrDateTime() + && !inner_type->isNumber()) + throw Exception("DataTypeWithDictionary is supported only for numbers, strings, Date or DateTime, but got " + + dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); +} + +void DataTypeWithDictionary::enumerateStreams(StreamCallback callback, SubstreamPath path) const +{ + path.push_back(Substream::DictionaryElements); + dictionary_type->enumerateStreams(callback, path); + path.back() = Substream::DictionaryIndexes; + indexes_type->enumerateStreams(callback, path); +} + +void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + OutputStreamGetter getter, + size_t offset, + size_t limit, + bool /*position_independent_encoding*/, + SubstreamPath path) const +{ + const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + + path.push_back(Substream::DictionaryElements); + if (auto stream = getter(path)) + { + if (offset == 0) + { + auto nested = column_with_dictionary.getUnique()->getNestedColumn(); + UInt64 nested_size = nested->size(); + writeIntBinary(nested_size, *stream); + dictionary_type->serializeBinaryBulk(*nested, *stream, 0, 0); + } + } + + path.back() = Substream::DictionaryIndexes; + if (auto stream = getter(path)) + indexes_type->serializeBinaryBulk(*column_with_dictionary.getIndexes(), *stream, offset, limit); +} + +void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( + IColumn & column, + InputStreamGetter getter, + size_t limit, + double /*avg_value_size_hint*/, + bool /*position_independent_encoding*/, + SubstreamPath path) const +{ + ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + + path.push_back(Substream::DictionaryElements); + if (ReadBuffer * stream = getter(path)) + { + if (column.empty()) + { + UInt64 nested_size; + readIntBinary(nested_size, *stream); + auto dict_column = column_with_dictionary.getUnique()->getNestedColumn()->cloneEmpty(); + dictionary_type->deserializeBinaryBulk(*dict_column, *stream, nested_size, 0); + + /// Note: it's assumed that rows inserted into columnUnique get incremental indexes. + column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, dict_column->size()); + } + } + + path.back() = Substream::DictionaryIndexes; + if (auto stream = getter(path)) + indexes_type->deserializeBinaryBulk(*column_with_dictionary.getIndexes(), *stream, limit, 0); +} + +void DataTypeWithDictionary::serializeBinary(const Field & field, WriteBuffer & ostr) const +{ + dictionary_type->serializeBinary(field, ostr); +} +void DataTypeWithDictionary::deserializeBinary(Field & field, ReadBuffer & istr) const +{ + dictionary_type->deserializeBinary(field, istr); +} + +template +void DataTypeWithDictionary::serializeImpl( + const IColumn & column, size_t row_num, WriteBuffer & ostr, + DataTypeWithDictionary::SerealizeFunctionPtr func, Args & ... args) const +{ + auto & column_with_dictionary = getColumnWithDictionary(column); + size_t unique_row_number = column_with_dictionary.getIndexes()->getUInt(row_num); + (dictionary_type.get()->*func)(*column_with_dictionary.getUnique(), unique_row_number, ostr, std::forward(args)...); +} + +template +void DataTypeWithDictionary::deserializeImpl( + IColumn & column, ReadBuffer & istr, + DataTypeWithDictionary::DeserealizeFunctionPtr func, Args ... args) const +{ + auto & column_with_dictionary = getColumnWithDictionary(column); + auto nested_unique = getNestedUniqueColumn(column_with_dictionary).assumeMutable(); + + auto size = column_with_dictionary.size(); + auto unique_size = nested_unique->size(); + + (dictionary_type.get()->*func)(*nested_unique, istr, std::forward(args)...); + + /// Note: Insertion into ColumnWithDictionary from it's nested column may cause insertion from column to itself. + /// Generally it's wrong because column may reallocate memory before insertion. + column_with_dictionary.insertFrom(*nested_unique, unique_size); + if (column_with_dictionary.getIndexes()->getUInt(size) != unique_size) + nested_unique->popBack(1); +} + +template +MutableColumnPtr DataTypeWithDictionary::createColumnImpl() const +{ + return ColumnWithDictionary::create(ColumnUnique::create(dictionary_type), + indexes_type->createColumn()); +} + +template +MutableColumnPtr DataTypeWithDictionary::createColumnImpl() const +{ + if (typeid_cast(indexes_type.get())) + return createColumnImpl(); + if (typeid_cast(indexes_type.get())) + return createColumnImpl(); + if (typeid_cast(indexes_type.get())) + return createColumnImpl(); + if (typeid_cast(indexes_type.get())) + return createColumnImpl(); + + throw Exception("The type of indexes must be unsigned integer, but got " + dictionary_type->getName(), + ErrorCodes::LOGICAL_ERROR); +} + +struct CreateColumnVector +{ + MutableColumnPtr & column; + const DataTypeWithDictionary * data_type_with_dictionary; + const IDataType * type; + + CreateColumnVector(MutableColumnPtr & column, const DataTypeWithDictionary * data_type_with_dictionary, + const IDataType * type) + : column(column), data_type_with_dictionary(data_type_with_dictionary), type(type) {} + + template + void operator()() + { + if (typeid_cast *>(type)) + column = data_type_with_dictionary->createColumnImpl>(); + } +}; + +MutableColumnPtr DataTypeWithDictionary::createColumn() const +{ + auto type = dictionary_type; + if (type->isNullable()) + type = static_cast(*dictionary_type).getNestedType(); + + if (type->isString()) + return createColumnImpl(); + if (type->isFixedString()) + return createColumnImpl(); + if (typeid_cast(type.get())) + return createColumnImpl>(); + if (typeid_cast(type.get())) + return createColumnImpl>(); + if (type->isNumber()) + { + MutableColumnPtr column; + TypeListNumbers::forEach(CreateColumnVector(column, this, dictionary_type.get())); + + if (!column) + throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR); + + return std::move(column); + } + + throw Exception("Unexpected dictionary type for DataTypeWithDictionary: " + type->getName(), + ErrorCodes::LOGICAL_ERROR); +} + +bool DataTypeWithDictionary::equals(const IDataType & rhs) const +{ + if (typeid(rhs) != typeid(*this)) + return false; + + auto & rhs_with_dictionary = static_cast(rhs); + return dictionary_type->equals(*rhs_with_dictionary.dictionary_type) + && indexes_type->equals(*rhs_with_dictionary.indexes_type); +} + + static DataTypePtr create(const ASTPtr & arguments) { diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index a86c89486f6..7206f741000 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -1,25 +1,9 @@ #pragma once -#include -#include -#include -#include #include -#include -#include -#include -#include -#include "DataTypeDate.h" -#include "DataTypeDateTime.h" namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} - class DataTypeWithDictionary : public IDataType { private: @@ -27,24 +11,7 @@ private: DataTypePtr indexes_type; public: - - DataTypeWithDictionary(DataTypePtr dictionary_type_, DataTypePtr indexes_type_) - : dictionary_type(std::move(dictionary_type_)), indexes_type(std::move(indexes_type_)) - { - if (!indexes_type->isUnsignedInteger()) - throw Exception("Index type of DataTypeWithDictionary must be unsigned integer, but got " - + indexes_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto inner_type = dictionary_type; - if (dictionary_type->isNullable()) - inner_type = static_cast(*dictionary_type).getNestedType(); - - if (!inner_type->isStringOrFixedString() - && !inner_type->isDateOrDateTime() - && !inner_type->isNumber()) - throw Exception("DataTypeWithDictionary is supported only for numbers, strings, Date or DateTime, but got " - + dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } + DataTypeWithDictionary(DataTypePtr dictionary_type_, DataTypePtr indexes_type_); const DataTypePtr & getDictionaryType() const { return dictionary_type; } const DataTypePtr & getIndexesType() const { return indexes_type; } @@ -55,13 +22,7 @@ public: } const char * getFamilyName() const override { return "WithDictionary"; } - void enumerateStreams(StreamCallback callback, SubstreamPath path) const override - { - path.push_back(Substream::DictionaryElements); - dictionary_type->enumerateStreams(callback, path); - path.back() = Substream::DictionaryIndexes; - indexes_type->enumerateStreams(callback, path); - } + void enumerateStreams(StreamCallback callback, SubstreamPath path) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, @@ -69,21 +30,7 @@ public: size_t offset, size_t limit, bool /*position_independent_encoding*/, - SubstreamPath path) const override - { - const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); - - path.push_back(Substream::DictionaryElements); - if (auto stream = getter(path)) - { - if (offset == 0) - dictionary_type->serializeBinaryBulk(*column_with_dictionary.getUnique()->getNestedColumn(), *stream, 0, 0); - } - - path.back() = Substream::DictionaryIndexes; - if (auto stream = getter(path)) - indexes_type->serializeBinaryBulk(*column_with_dictionary.getIndexes(), *stream, offset, limit); - } + SubstreamPath path) const override; void deserializeBinaryBulkWithMultipleStreams( IColumn & column, @@ -91,77 +38,10 @@ public: size_t limit, double /*avg_value_size_hint*/, bool /*position_independent_encoding*/, - SubstreamPath path) const override - { - ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + SubstreamPath path) const override; - path.push_back(Substream::DictionaryElements); - if (ReadBuffer * stream = getter(path)) - { - if (column.empty()) - { - auto dict_column = column_with_dictionary.getUnique()->getNestedColumn()->cloneEmpty(); - dictionary_type->deserializeBinaryBulk(*dict_column, *stream, 0, 0); - - /// Note: it's assumed that rows inserted into columnUnique get incremental indexes. - column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, dict_column->size()); - } - } - - path.back() = Substream::DictionaryIndexes; - if (auto stream = getter(path)) - indexes_type->deserializeBinaryBulk(*column_with_dictionary.getIndexes(), *stream, limit, 0); - } - - void serializeBinary(const Field & field, WriteBuffer & ostr) const override { dictionary_type->serializeBinary(field, ostr); } - void deserializeBinary(Field & field, ReadBuffer & istr) const override { dictionary_type->deserializeBinary(field, istr); } - - const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column) const - { - return typeid_cast(column);; - } - - ColumnWithDictionary & getColumnWithDictionary(IColumn & column) const - { - return typeid_cast(column);; - } - - IColumn & getNestedUniqueColumn(ColumnWithDictionary & column_with_dictionary) const - { - return column_with_dictionary.getUnique()->getNestedColumn()->assumeMutableRef(); - } - - template - using SerealizeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, WriteBuffer &, Args & ...) const; - - template - void serializeImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, SerealizeFunctionPtr func, Args & ... args) const - { - auto & column_with_dictionary = getColumnWithDictionary(column); - size_t unique_row_number = column_with_dictionary.getIndexes()->getUInt(row_num); - (dictionary_type.get()->*func)(*column_with_dictionary.getUnique(), unique_row_number, ostr, std::forward(args)...); - } - - template - using DeserealizeFunctionPtr = void (IDataType::*)(IColumn &, ReadBuffer &, Args ...) const; - - template - void deserializeImpl(IColumn & column, ReadBuffer & istr, DeserealizeFunctionPtr func, Args ... args) const - { - auto & column_with_dictionary = getColumnWithDictionary(column); - auto nested_unique = getNestedUniqueColumn(column_with_dictionary).assumeMutable(); - - auto size = column_with_dictionary.size(); - auto unique_size = nested_unique->size(); - - (dictionary_type.get()->*func)(*nested_unique, istr, std::forward(args)...); - - /// Note: Insertion into ColumnWithDictionary from it's nested column may cause insertion from column to itself. - /// Generally it's wrong because column may reallocate memory before insertion. - column_with_dictionary.insertFrom(*nested_unique, unique_size); - if (column_with_dictionary.getIndexes()->getUInt(size) != unique_size) - nested_unique->popBack(1); - } + void serializeBinary(const Field & field, WriteBuffer & ostr) const override; + void deserializeBinary(Field & field, ReadBuffer & istr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { @@ -221,88 +101,11 @@ public: serializeImpl(column, row_num, ostr, &IDataType::serializeTextXML); } - template - MutableColumnPtr createColumnImpl() const - { - return ColumnWithDictionary::create(ColumnUnique::create(dictionary_type), - indexes_type->createColumn()); - } - - template - MutableColumnPtr createColumnImpl() const - { - if (typeid_cast(indexes_type.get())) - return createColumnImpl(); - if (typeid_cast(indexes_type.get())) - return createColumnImpl(); - if (typeid_cast(indexes_type.get())) - return createColumnImpl(); - if (typeid_cast(indexes_type.get())) - return createColumnImpl(); - - throw Exception("The type of indexes must be unsigned integer, but got " + dictionary_type->getName(), - ErrorCodes::LOGICAL_ERROR); - } - -private: - struct CreateColumnVector - { - MutableColumnPtr & column; - const DataTypeWithDictionary * data_type_with_dictionary; - const IDataType * type; - - CreateColumnVector(MutableColumnPtr & column, const DataTypeWithDictionary * data_type_with_dictionary, - const IDataType * type) - : column(column), data_type_with_dictionary(data_type_with_dictionary), type(type) {} - - template - void operator()() - { - if (typeid_cast *>(type)) - column = data_type_with_dictionary->createColumnImpl>(); - } - }; - -public: - MutableColumnPtr createColumn() const override - { - auto type = dictionary_type; - if (type->isNullable()) - type = static_cast(*dictionary_type).getNestedType(); - - if (type->isString()) - return createColumnImpl(); - if (type->isFixedString()) - return createColumnImpl(); - if (typeid_cast(type.get())) - return createColumnImpl>(); - if (typeid_cast(type.get())) - return createColumnImpl>(); - if (type->isNumber()) - { - MutableColumnPtr column; - TypeListNumbers::forEach(CreateColumnVector(column, this, dictionary_type.get())); - - if (!column) - throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR); - - return std::move(column); - } - - throw Exception("Unexpected dictionary type for DataTypeWithDictionary: " + type->getName(), ErrorCodes::LOGICAL_ERROR); - } + MutableColumnPtr createColumn() const override; Field getDefault() const override { return dictionary_type->getDefault(); } - bool equals(const IDataType & rhs) const override - { - if (typeid(rhs) != typeid(*this)) - return false; - - auto & rhs_with_dictionary = static_cast(rhs); - return dictionary_type->equals(*rhs_with_dictionary.dictionary_type) - && indexes_type->equals(*rhs_with_dictionary.indexes_type); - } + bool equals(const IDataType & rhs) const override; bool isParametric() const override { return true; } bool haveSubtypes() const override { return true; } @@ -332,6 +135,44 @@ public: bool isNullable() const override { return false; } bool onlyNull() const override { return false; } bool withDictionary() const override { return true; } + +private: + const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column) const + { + return typeid_cast(column);; + } + + ColumnWithDictionary & getColumnWithDictionary(IColumn & column) const + { + return typeid_cast(column);; + } + + IColumn & getNestedUniqueColumn(ColumnWithDictionary & column_with_dictionary) const + { + return column_with_dictionary.getUnique()->getNestedColumn()->assumeMutableRef(); + } + + template + using SerealizeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, WriteBuffer &, Args & ...) const; + + template + void serializeImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, + SerealizeFunctionPtr func, Args & ... args) const; + + template + using DeserealizeFunctionPtr = void (IDataType::*)(IColumn &, ReadBuffer &, Args ...) const; + + template + void deserializeImpl(IColumn & column, ReadBuffer & istr, + DeserealizeFunctionPtr func, Args ... args) const; + + template + MutableColumnPtr createColumnImpl() const; + + template + MutableColumnPtr createColumnImpl() const; + + friend class CreateColumnVector; }; } From ed83798dd2f0ea108e28fa6bd90effe150884b13 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 18:17:10 +0300 Subject: [PATCH 033/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 18 ++++ dbms/src/DataTypes/DataTypeWithDictionary.h | 14 --- dbms/src/Functions/FunctionsMiscellaneous.cpp | 99 +++++++++++++++++++ dbms/src/Functions/FunctionsMiscellaneous.h | 96 ------------------ 4 files changed, 117 insertions(+), 110 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index bcd0571ff5e..5205abfca05 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -20,6 +20,24 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column) const + { + return typeid_cast(column); + } + + ColumnWithDictionary & getColumnWithDictionary(IColumn & column) const + { + return typeid_cast(column); + } + + IColumn & getNestedUniqueColumn(ColumnWithDictionary & column_with_dictionary) const + { + return column_with_dictionary.getUnique()->getNestedColumn()->assumeMutableRef(); + } +} + DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_, DataTypePtr indexes_type_) : dictionary_type(std::move(dictionary_type_)), indexes_type(std::move(indexes_type_)) { diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index 7206f741000..f8dc96e4669 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -137,20 +137,6 @@ public: bool withDictionary() const override { return true; } private: - const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column) const - { - return typeid_cast(column);; - } - - ColumnWithDictionary & getColumnWithDictionary(IColumn & column) const - { - return typeid_cast(column);; - } - - IColumn & getNestedUniqueColumn(ColumnWithDictionary & column_with_dictionary) const - { - return column_with_dictionary.getUnique()->getNestedColumn()->assumeMutableRef(); - } template using SerealizeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, WriteBuffer &, Args & ...) const; diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index c15614d27a8..7a74ea8d046 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -23,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -1833,6 +1835,103 @@ public: }; +class FunctionMakeDictionary: public IFunction +{ +public: + static constexpr auto name = "makeDictionary"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return std::make_shared(arguments[0], std::make_shared()); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + auto column = res.type->createColumn(); + column->insertRangeFrom(*arg.column, 0, arg.column->size()); + res.column = std::move(column); + } +}; + +class FunctionDictionaryIndexes: public IFunction +{ +public: + static constexpr auto name = "dictionaryIndexes"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + auto * type = typeid_cast(arguments[0].get()); + if (!type) + throw Exception("First first argument of function dictionaryIndexes must be ColumnWithDictionary, but got" + + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getIndexesType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + res.column = typeid_cast(arg.column.get())->getIndexesPtr(); + } +}; + +class FunctionDictionaryValues: public IFunction +{ +public: + static constexpr auto name = "dictionaryValues"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + auto * type = typeid_cast(arguments[0].get()); + if (!type) + throw Exception("First first argument of function dictionaryValues must be ColumnWithDictionary, but got" + + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getDictionaryType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + const auto * column_with_dictionary = typeid_cast(arg.column.get()); + res.column = column_with_dictionary->getUnique()->getNestedColumn()->cloneResized(arg.column->size()); + } +}; + + std::string FunctionVersion::getVersion() const { std::ostringstream os; diff --git a/dbms/src/Functions/FunctionsMiscellaneous.h b/dbms/src/Functions/FunctionsMiscellaneous.h index 47d355d9d1a..1b3d1f12fdd 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.h +++ b/dbms/src/Functions/FunctionsMiscellaneous.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB { @@ -37,101 +36,6 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; }; -class FunctionMakeDictionary: public IFunction -{ -public: - static constexpr auto name = "makeDictionary"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - return std::make_shared(arguments[0], std::make_shared()); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override - { - auto arg_num = arguments[0]; - const auto & arg = block.getByPosition(arg_num); - auto & res = block.getByPosition(result); - auto column = res.type->createColumn(); - column->insertRangeFrom(*arg.column, 0, arg.column->size()); - res.column = std::move(column); - } -}; - -class FunctionDictionaryIndexes: public IFunction -{ -public: - static constexpr auto name = "dictionaryIndexes"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - auto * type = typeid_cast(arguments[0].get()); - if (!type) - throw Exception("First first argument of function dictionaryIndexes must be ColumnWithDictionary, but got" - + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return type->getIndexesType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override - { - auto arg_num = arguments[0]; - const auto & arg = block.getByPosition(arg_num); - auto & res = block.getByPosition(result); - res.column = typeid_cast(arg.column.get())->getIndexesPtr(); - } -}; - -class FunctionDictionaryValues: public IFunction -{ -public: - static constexpr auto name = "dictionaryValues"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - auto * type = typeid_cast(arguments[0].get()); - if (!type) - throw Exception("First first argument of function dictionaryValues must be ColumnWithDictionary, but got" - + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return type->getDictionaryType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override - { - auto arg_num = arguments[0]; - const auto & arg = block.getByPosition(arg_num); - auto & res = block.getByPosition(result); - const auto * column_with_dictionary = typeid_cast(arg.column.get()); - res.column = column_with_dictionary->getUnique()->getNestedColumn()->cloneResized(arg.column->size()); - } -}; /// Executes expression. Uses for lambda functions implementation. Can't be created from factory. class FunctionExpression : public IFunctionBase, public IPreparedFunction, From 153fda001e54c3549507aba2f102bcb5977a746c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 18:20:58 +0300 Subject: [PATCH 034/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 6 +++--- dbms/src/Functions/IFunction.cpp | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 5205abfca05..3402a920053 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -22,17 +22,17 @@ namespace ErrorCodes namespace { - const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column) const + const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column) { return typeid_cast(column); } - ColumnWithDictionary & getColumnWithDictionary(IColumn & column) const + ColumnWithDictionary & getColumnWithDictionary(IColumn & column) { return typeid_cast(column); } - IColumn & getNestedUniqueColumn(ColumnWithDictionary & column_with_dictionary) const + IColumn & getNestedUniqueColumn(ColumnWithDictionary & column_with_dictionary) { return column_with_dictionary.getUnique()->getNestedColumn()->assumeMutableRef(); } diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 810ce989b71..dbf9136fc04 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB From 99602c16333179a35176b7f00763797b4fe314eb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 18:56:57 +0300 Subject: [PATCH 035/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 18 ++++-------------- 1 file changed, 4 insertions(+), 14 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 3402a920053..f5b08961977 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -31,11 +31,6 @@ namespace { return typeid_cast(column); } - - IColumn & getNestedUniqueColumn(ColumnWithDictionary & column_with_dictionary) - { - return column_with_dictionary.getUnique()->getNestedColumn()->assumeMutableRef(); - } } DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_, DataTypePtr indexes_type_) @@ -137,7 +132,7 @@ void DataTypeWithDictionary::serializeImpl( { auto & column_with_dictionary = getColumnWithDictionary(column); size_t unique_row_number = column_with_dictionary.getIndexes()->getUInt(row_num); - (dictionary_type.get()->*func)(*column_with_dictionary.getUnique(), unique_row_number, ostr, std::forward(args)...); + (dictionary_type.get()->*func)(*column_with_dictionary.getUnique()->getNestedColumn(), unique_row_number, ostr, std::forward(args)...); } template @@ -146,18 +141,13 @@ void DataTypeWithDictionary::deserializeImpl( DataTypeWithDictionary::DeserealizeFunctionPtr func, Args ... args) const { auto & column_with_dictionary = getColumnWithDictionary(column); - auto nested_unique = getNestedUniqueColumn(column_with_dictionary).assumeMutable(); + auto temp_column = column_with_dictionary.getUnique()->cloneEmpty(); - auto size = column_with_dictionary.size(); - auto unique_size = nested_unique->size(); - - (dictionary_type.get()->*func)(*nested_unique, istr, std::forward(args)...); + (dictionary_type.get()->*func)(*temp_column, istr, std::forward(args)...); /// Note: Insertion into ColumnWithDictionary from it's nested column may cause insertion from column to itself. /// Generally it's wrong because column may reallocate memory before insertion. - column_with_dictionary.insertFrom(*nested_unique, unique_size); - if (column_with_dictionary.getIndexes()->getUInt(size) != unique_size) - nested_unique->popBack(1); + column_with_dictionary.insertFrom(*temp_column, 0); } template From 5d1c25df25130bd76822e302145640dbb79d146f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 19:28:14 +0300 Subject: [PATCH 036/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/Columns/ColumnUnique.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index df5d94d905a..c44b8e4aec7 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -206,6 +206,8 @@ void ColumnUnique::buildIndex() auto column = getRawColumnPtr(); index = std::make_unique(); + (*index)[StringRefWrapper(column, getDefaultValueIndex())] = getDefaultValueIndex(); + for (auto row : ext::range(numSpecialValues(), column->size())) { (*index)[StringRefWrapper(column, row)] = row; @@ -252,6 +254,9 @@ size_t ColumnUnique::uniqueInsert(const Field & x) template size_t ColumnUnique::uniqueInsertFrom(const IColumn & src, size_t n) { + if (is_nullable && src.isNullAt(n)) + return getNullValueIndex(); + auto ref = src.getDataAt(n); return uniqueInsertData(ref.data, ref.size); } From 7788d30dcaadb72a2313a86ab43ca5e2fda2864e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 20:23:18 +0300 Subject: [PATCH 037/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/Columns/ColumnUnique.h | 4 +--- dbms/src/Columns/ColumnVector.cpp | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index c44b8e4aec7..4c61bbe9d47 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -158,7 +158,7 @@ ColumnUnique::ColumnUnique(const DataTypePtr & type) : is auto & column_nullable = static_cast(nullable_column->assumeMutableRef()); column_holder = column_nullable.getNestedColumnPtr(); nullable_column_map = &column_nullable.getNullMapData(); - (*nullable_column_map)[1] = 0; + (*nullable_column_map)[getDefaultValueIndex()] = 0; } else column_holder = type->createColumn()->cloneResized(numSpecialValues()); @@ -206,8 +206,6 @@ void ColumnUnique::buildIndex() auto column = getRawColumnPtr(); index = std::make_unique(); - (*index)[StringRefWrapper(column, getDefaultValueIndex())] = getDefaultValueIndex(); - for (auto row : ext::range(numSpecialValues(), column->size())) { (*index)[StringRefWrapper(column, row)] = row; diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 582e3a0afa5..63b30bf4add 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -118,7 +118,7 @@ MutableColumnPtr ColumnVector::cloneResized(size_t size) const memcpy(&new_col.data[0], &data[0], count * sizeof(data[0])); if (size > count) - memset(&new_col.data[count], static_cast(value_type()), size - count); + memset(&new_col.data[count], static_cast(value_type()), (size - count) * sizeof(value_type())); } return std::move(res); From e2416cfa6f12adc572fe500d8ecbe1f6b7629c77 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 20:25:10 +0300 Subject: [PATCH 038/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/Columns/ColumnVector.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 63b30bf4add..8aa9f463c52 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -118,7 +118,7 @@ MutableColumnPtr ColumnVector::cloneResized(size_t size) const memcpy(&new_col.data[0], &data[0], count * sizeof(data[0])); if (size > count) - memset(&new_col.data[count], static_cast(value_type()), (size - count) * sizeof(value_type())); + memset(&new_col.data[count], static_cast(value_type()), (size - count) * sizeof(value_type)); } return std::move(res); From ec46b915e8d4eaabea97223f5be0a21f14b55eff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 20:34:17 +0300 Subject: [PATCH 039/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index f5b08961977..d0d8e058f12 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -208,7 +208,7 @@ MutableColumnPtr DataTypeWithDictionary::createColumn() const if (type->isNumber()) { MutableColumnPtr column; - TypeListNumbers::forEach(CreateColumnVector(column, this, dictionary_type.get())); + TypeListNumbers::forEach(CreateColumnVector(column, this, type.get())); if (!column) throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR); From 5e076ec594d95a2e9bd7cd284d9133d3d47fb461 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 20:46:12 +0300 Subject: [PATCH 040/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index d0d8e058f12..69d7473f338 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -64,7 +64,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( OutputStreamGetter getter, size_t offset, size_t limit, - bool /*position_independent_encoding*/, + bool position_independent_encoding, SubstreamPath path) const { const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); @@ -77,7 +77,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( auto nested = column_with_dictionary.getUnique()->getNestedColumn(); UInt64 nested_size = nested->size(); writeIntBinary(nested_size, *stream); - dictionary_type->serializeBinaryBulk(*nested, *stream, 0, 0); + dictionary_type->serializeBinaryBulkWithMultipleStreams(*nested, getter, 0, 0, position_independent_encoding, path); } } @@ -91,7 +91,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( InputStreamGetter getter, size_t limit, double /*avg_value_size_hint*/, - bool /*position_independent_encoding*/, + bool position_independent_encoding, SubstreamPath path) const { ColumnWithDictionary & column_with_dictionary = typeid_cast(column); @@ -104,7 +104,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( UInt64 nested_size; readIntBinary(nested_size, *stream); auto dict_column = column_with_dictionary.getUnique()->getNestedColumn()->cloneEmpty(); - dictionary_type->deserializeBinaryBulk(*dict_column, *stream, nested_size, 0); + dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, *stream, nested_size, 0, position_independent_encoding, path); /// Note: it's assumed that rows inserted into columnUnique get incremental indexes. column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, dict_column->size()); From 040c5acc35f72b1098b5f194668157895edd4348 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 May 2018 20:47:29 +0300 Subject: [PATCH 041/315] Moved DataTypeWithDictionary implementation to DataTypeWithDictionary.cpp --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 69d7473f338..cf7f10dd681 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -104,7 +104,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( UInt64 nested_size; readIntBinary(nested_size, *stream); auto dict_column = column_with_dictionary.getUnique()->getNestedColumn()->cloneEmpty(); - dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, *stream, nested_size, 0, position_independent_encoding, path); + dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, getter, nested_size, 0, position_independent_encoding, path); /// Note: it's assumed that rows inserted into columnUnique get incremental indexes. column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, dict_column->size()); From 9c696f40b90962a6e2686ceb9f740698a65e7da3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 13:48:09 +0300 Subject: [PATCH 042/315] Fixed ColumnWithDictionsry insert functions. Added insertFromFullColumn and insertRangeFromFullColumn. --- dbms/src/Columns/ColumnWithDictionary.h | 28 +++++++++++++++++-- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 4 +-- dbms/src/Functions/IFunction.cpp | 17 +++-------- 3 files changed, 31 insertions(+), 18 deletions(-) diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index 38fffebda58..dbb89c74ec1 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -72,12 +72,36 @@ public: } void insert(const Field & x) override { getIndexes()->insert(Field(UInt64(getUnique()->uniqueInsert(x)))); } - void insertFrom(const IColumn & src, size_t n) override { getIndexes()->insert(getUnique()->uniqueInsertFrom(src, n)); } - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override + + void insertFromFullColumn(const IColumn & src, size_t n) + { + getIndexes()->insert(getUnique()->uniqueInsertFrom(src, n)); + } + void insertFrom(const IColumn & src, size_t n) override + { + if (!typeid_cast(&src)) + throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); + auto & src_with_dict = static_cast(src); + size_t idx = src_with_dict.getIndexes()->getUInt(n); + insertFromFullColumn(*src_with_dict.getUnique()->getNestedColumn(), idx); + } + + void insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length) { auto inserted_indexes = getUnique()->uniqueInsertRangeFrom(src, start, length); getIndexes()->insertRangeFrom(*inserted_indexes, 0, length); } + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override + { + if (!typeid_cast(&src)) + throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); + + auto & src_with_dict = static_cast(src); + auto & src_nested = src_with_dict.getUnique()->getNestedColumn(); + auto inserted_idx = getUnique()->uniqueInsertRangeFrom(*src_nested, 0, src_nested->size()); + auto idx = inserted_idx->index(src_with_dict.getIndexes()->cut(start, length), 0); + getIndexes()->insertRangeFrom(*idx, 0, length); + } void insertData(const char * pos, size_t length) override { diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index cf7f10dd681..1ede81241ff 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -145,9 +145,7 @@ void DataTypeWithDictionary::deserializeImpl( (dictionary_type.get()->*func)(*temp_column, istr, std::forward(args)...); - /// Note: Insertion into ColumnWithDictionary from it's nested column may cause insertion from column to itself. - /// Generally it's wrong because column may reallocate memory before insertion. - column_with_dictionary.insertFrom(*temp_column, 0); + column_with_dictionary.insertFromFullColumn(*temp_column, 0); } template diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index dbf9136fc04..c15eca35e90 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -306,24 +306,15 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si executeWithoutColumnsWithDictionary(temp_block, temp_numbers, 0); auto & temp_res_col = temp_block.getByPosition(0).column; auto & res_col = block.getByPosition(result); - res_col.column = res_col.type->createColumn(); + auto col_wit_dict_ptr = res_col.type->createColumn(); - auto * col_with_dict = checkAndGetColumn(res_col.column.get()); + auto * col_with_dict = typeid_cast(col_wit_dict_ptr.get()); if (!col_with_dict) throw Exception("Expected ColumnWithDictionary, got" + res_col.column->getName(), ErrorCodes::LOGICAL_ERROR); - auto & mut_col_with_dict = const_cast(*col_with_dict); - - if (indexes) - { - auto new_ind = mut_col_with_dict.getUnique()->uniqueInsertRangeFrom(*temp_res_col, 0, temp_res_col->size()); - mut_col_with_dict.setIndexes(new_ind->index(indexes, 0)->assumeMutable()); - } - else - { - mut_col_with_dict.insertRangeFrom(*temp_res_col, 0, temp_res_col->size()); - } + col_with_dict->insertRangeFromFullColumn(*temp_res_col, 0, temp_res_col->size()); + res_col.column = indexes ? col_with_dict->index(indexes, 0) : std::move(col_wit_dict_ptr); return; } } From c61049dc59def52aed0703c9aba38ee9e95fd9f5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 13:58:28 +0300 Subject: [PATCH 043/315] Fixed ColumnWithDictionsry insert functions. Added insertFromFullColumn and insertRangeFromFullColumn. --- dbms/src/Functions/FunctionsMiscellaneous.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index 7a74ea8d046..b1fcf66ba7e 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -1859,7 +1859,7 @@ public: const auto & arg = block.getByPosition(arg_num); auto & res = block.getByPosition(result); auto column = res.type->createColumn(); - column->insertRangeFrom(*arg.column, 0, arg.column->size()); + typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); res.column = std::move(column); } }; From cf60dba41434a3787207046c1a0d868e135680a8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 13:59:19 +0300 Subject: [PATCH 044/315] Fixed ColumnWithDictionsry insert functions. Added insertFromFullColumn and insertRangeFromFullColumn. --- dbms/src/Functions/FunctionsMiscellaneous.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index b1fcf66ba7e..ec728b0b275 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -1859,7 +1859,7 @@ public: const auto & arg = block.getByPosition(arg_num); auto & res = block.getByPosition(result); auto column = res.type->createColumn(); - typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); + typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); res.column = std::move(column); } }; From 530b99cee55de33681584abccd4e228ec6e39373 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 14:39:15 +0300 Subject: [PATCH 045/315] Added ConvertColumnWithDictionaryToFullBlockInputStream. --- ...lumnWithDictionaryToFullBlockInputStream.h | 56 +++++++++++++++++++ .../Interpreters/InterpreterSelectQuery.cpp | 4 +- 2 files changed, 59 insertions(+), 1 deletion(-) create mode 100644 dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h diff --git a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h new file mode 100644 index 00000000000..37d3a7c762e --- /dev/null +++ b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + + +/** Combines several sources into one. + * Unlike UnionBlockInputStream, it does this sequentially. + * Blocks of different sources are not interleaved with each other. + */ +class ConvertColumnWithDictionaryToFullBlockInputStream : public IProfilingBlockInputStream +{ +public: + ConvertColumnWithDictionaryToFullBlockInputStream(BlockInputStreamPtr input) + { + children.push_bcak(input); + } + + String getName() const override { return "ConvertColumnWithDictionaryToFull"; } + + Block getHeader() const override { return children.at(0)->getHeader(); } + +protected: + Block readImpl() override + { + Block res; + + for (auto & column : res) + { + auto * type_with_dict = typeid_cast(column.type.get()); + auto * col_with_dict = typeid_cast(column.column.get()); + + if (type_with_dict && !col_with_dict) + throw Exception("Invalid column for " + type_with_dict->getName() + ": " + column.column->getName(), + ErrorCodes::LOGICAL_ERROR); + + if (!type_with_dict && col_with_dict) + throw Exception("Invalid type for " + col_with_dict->getName() + ": " + column.type->getName(), + ErrorCodes::LOGICAL_ERROR); + + if (type_with_dict && col_with_dict) + { + column.column = col_with_dict->convertToFullColumn(); + column.type = type_with_dict->getDictionaryType(); + } + } + + return res; + } +}; + +} diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 7ed250e9036..ac45befce37 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -751,7 +752,8 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre { pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, expression); + stream = std::make_shared( + std::make_shared(stream, expression)); }); Names key_names; From 13ba167dce33b9b50b78cdc2873b2bb9c20d287e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 14:40:36 +0300 Subject: [PATCH 046/315] Added ConvertColumnWithDictionaryToFullBlockInputStream. --- .../ConvertColumnWithDictionaryToFullBlockInputStream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h index 37d3a7c762e..113b19256e4 100644 --- a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h +++ b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h @@ -17,7 +17,7 @@ class ConvertColumnWithDictionaryToFullBlockInputStream : public IProfilingBlock public: ConvertColumnWithDictionaryToFullBlockInputStream(BlockInputStreamPtr input) { - children.push_bcak(input); + children.push_back(input); } String getName() const override { return "ConvertColumnWithDictionaryToFull"; } From f4627af23fbdc096e0213cdfc1e2dac24a74e2c6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 14:44:52 +0300 Subject: [PATCH 047/315] Added ConvertColumnWithDictionaryToFullBlockInputStream. --- .../ConvertColumnWithDictionaryToFullBlockInputStream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h index 113b19256e4..86799783449 100644 --- a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h +++ b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h @@ -27,7 +27,7 @@ public: protected: Block readImpl() override { - Block res; + Block res = children.back()->read(); for (auto & column : res) { From 67769ef333b70fbdce871b988356f8f7a0d2a5fe Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 14:47:39 +0300 Subject: [PATCH 048/315] Added ConvertColumnWithDictionaryToFullBlockInputStream. --- .../ConvertColumnWithDictionaryToFullBlockInputStream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h index 86799783449..5abde1323a3 100644 --- a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h +++ b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h @@ -15,7 +15,7 @@ namespace DB class ConvertColumnWithDictionaryToFullBlockInputStream : public IProfilingBlockInputStream { public: - ConvertColumnWithDictionaryToFullBlockInputStream(BlockInputStreamPtr input) + ConvertColumnWithDictionaryToFullBlockInputStream(const BlockInputStreamPtr & input) { children.push_back(input); } From af5475b77c3ac07841b0692ee6ec058d5a2552ad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 14:50:42 +0300 Subject: [PATCH 049/315] Added ConvertColumnWithDictionaryToFullBlockInputStream. --- ...rtColumnWithDictionaryToFullBlockInputStream.h | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h index 5abde1323a3..784850feb6a 100644 --- a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h +++ b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h @@ -15,21 +15,22 @@ namespace DB class ConvertColumnWithDictionaryToFullBlockInputStream : public IProfilingBlockInputStream { public: - ConvertColumnWithDictionaryToFullBlockInputStream(const BlockInputStreamPtr & input) + explicit ConvertColumnWithDictionaryToFullBlockInputStream(const BlockInputStreamPtr & input) { children.push_back(input); } String getName() const override { return "ConvertColumnWithDictionaryToFull"; } - Block getHeader() const override { return children.at(0)->getHeader(); } + Block getHeader() const override { return convert(children.at(0)->getHeader()); } protected: - Block readImpl() override - { - Block res = children.back()->read(); + Block readImpl() override { return convert(children.back()->read()); } - for (auto & column : res) +private: + Block convert(Block && block) const + { + for (auto & column : block) { auto * type_with_dict = typeid_cast(column.type.get()); auto * col_with_dict = typeid_cast(column.column.get()); @@ -49,7 +50,7 @@ protected: } } - return res; + return std::move(block); } }; From 0897d2a9c4e2e34fa324615da6dc2e6b40efca1d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 15:11:57 +0300 Subject: [PATCH 050/315] Added ConvertColumnWithDictionaryToFullBlockInputStream. --- .../AggregateFunctionFactory.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index eca854a031b..8a7c5443f27 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -13,6 +13,7 @@ #include #include +#include namespace DB @@ -41,6 +42,20 @@ void AggregateFunctionFactory::registerFunction(const String & name, Creator cre ErrorCodes::LOGICAL_ERROR); } +static DataTypes convertTypesWithDictionaryToNested(const DataTypes & types) +{ + DataTypes res_types; + res_types.reserve(types.size()); + for (const auto & type : types) + { + if (auto * type_with_dict = typeid_cast(type.get())) + res_types.push_back(type_with_dict->getDictionaryType()); + else + res_types.push_back(type); + } + + return std::move(res_types); +} AggregateFunctionPtr AggregateFunctionFactory::get( const String & name, @@ -57,7 +72,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( if (!combinator) throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR); - DataTypes nested_types = combinator->transformArguments(argument_types); + DataTypes nested_types = combinator->transformArguments(convertTypesWithDictionaryToNested(argument_types)); AggregateFunctionPtr nested_function; From 535e883770e3a55f0e1bcee32df235ba445be54d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 15:17:53 +0300 Subject: [PATCH 051/315] Added ConvertColumnWithDictionaryToFullBlockInputStream. --- dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 8a7c5443f27..0bfbc5e7a58 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -63,6 +63,8 @@ AggregateFunctionPtr AggregateFunctionFactory::get( const Array & parameters, int recursion_level) const { + auto type_without_dictionary = convertTypesWithDictionaryToNested(argument_types); + /// If one of types is Nullable, we apply aggregate function combinator "Null". if (std::any_of(argument_types.begin(), argument_types.end(), @@ -72,7 +74,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( if (!combinator) throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR); - DataTypes nested_types = combinator->transformArguments(convertTypesWithDictionaryToNested(argument_types)); + DataTypes nested_types = combinator->transformArguments(type_without_dictionary); AggregateFunctionPtr nested_function; @@ -85,7 +87,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( return combinator->transformAggregateFunction(nested_function, argument_types, parameters); } - auto res = getImpl(name, argument_types, parameters, recursion_level); + auto res = getImpl(name, type_without_dictionary, parameters, recursion_level); if (!res) throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR); return res; From 310931a8a575479070401bb9577809229d464f5d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 19:28:53 +0300 Subject: [PATCH 052/315] added DataTypeWithDictionary serialization per granule --- dbms/src/Columns/ColumnsCommon.cpp | 33 +++++++++++ dbms/src/Columns/ColumnsCommon.h | 4 ++ dbms/src/DataTypes/DataTypeWithDictionary.cpp | 56 +++++++++++-------- dbms/src/DataTypes/IDataType.cpp | 2 +- dbms/src/DataTypes/IDataType.h | 2 +- 5 files changed, 72 insertions(+), 25 deletions(-) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index db6abde09ac..72ce9d0fcae 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -326,6 +327,38 @@ namespace detail template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); + + template + MutableColumnPtr getUniqueIndexImpl(const PaddedPODArray & index) + { + HashSet hash_table; + for (auto val : index.getData()) + hash_table.insert(val); + + auto res_col = ColumnVector::create(); + auto & data = res_col->getData(); + + data.reserve(hash_table.size()); + for (auto val : hash_table) + data.push_back(val); + + return std::move(res_col); + } +} + +MutableColumnPtr getUniqueIndex(const ColumnPtr & column) +{ + if (auto * data_uint8 = detail::getIndexesData(column)) + return detail::getUniqueIndexImpl(*data_uint8); + else if (auto * data_uint16 = detail::getIndexesData(column)) + return detail::getUniqueIndexImpl(*data_uint16); + else if (auto * data_uint32 = detail::getIndexesData(column)) + return detail::getUniqueIndexImpl(*data_uint32); + else if (auto * data_uint64 = detail::getIndexesData(column)) + return detail::getUniqueIndexImpl(*data_uint64); + else + throw Exception("Indexes column for getUniqueIndex must be ColumnUInt, got" + column->getName(), + ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/Columns/ColumnsCommon.h b/dbms/src/Columns/ColumnsCommon.h index 8ef90f48625..06d3dcbd023 100644 --- a/dbms/src/Columns/ColumnsCommon.h +++ b/dbms/src/Columns/ColumnsCommon.h @@ -73,4 +73,8 @@ ColumnPtr selectIndexImpl(const Column & column, const ColumnPtr & indexes, size template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; + +/// Get unique values from index column (ColumnUInt*). +MutableColumnPtr getUniqueIndex(const ColumnPtr & column); + } diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 1ede81241ff..77164cdc522 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -53,7 +54,7 @@ DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_, Dat void DataTypeWithDictionary::enumerateStreams(StreamCallback callback, SubstreamPath path) const { - path.push_back(Substream::DictionaryElements); + path.push_back(Substream::DictionaryKeys); dictionary_type->enumerateStreams(callback, path); path.back() = Substream::DictionaryIndexes; indexes_type->enumerateStreams(callback, path); @@ -68,22 +69,28 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( SubstreamPath path) const { const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + const auto & indexes = column_with_dictionary.getIndexesPtr(); + const auto & keys = column_with_dictionary.getUnique()->getNestedColumn(); - path.push_back(Substream::DictionaryElements); + path.push_back(Substream::DictionaryKeys); if (auto stream = getter(path)) { - if (offset == 0) - { - auto nested = column_with_dictionary.getUnique()->getNestedColumn(); - UInt64 nested_size = nested->size(); - writeIntBinary(nested_size, *stream); - dictionary_type->serializeBinaryBulkWithMultipleStreams(*nested, getter, 0, 0, position_independent_encoding, path); - } - } + bool full_column = offset == 0 && limit >= indexes->size(); + + auto unique_indexes = getUniqueIndex(full_column ? indexes : indexes->cut(offset, limit)); + auto used_keys = keys->index(*unique_indexes, 0); + + UInt64 used_keys_size = used_keys->size(); + writeIntBinary(used_keys_size, *stream); + dictionary_type->serializeBinaryBulkWithMultipleStreams(*keys, getter, 0, 0, + position_independent_encoding, path); + } path.back() = Substream::DictionaryIndexes; if (auto stream = getter(path)) - indexes_type->serializeBinaryBulk(*column_with_dictionary.getIndexes(), *stream, offset, limit); + { + indexes_type->serializeBinaryBulk(*indexes, *stream, offset, limit); + } } void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( @@ -95,25 +102,28 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( SubstreamPath path) const { ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + ColumnPtr indexes; - path.push_back(Substream::DictionaryElements); + path.push_back(Substream::DictionaryKeys); if (ReadBuffer * stream = getter(path)) { - if (column.empty()) - { - UInt64 nested_size; - readIntBinary(nested_size, *stream); - auto dict_column = column_with_dictionary.getUnique()->getNestedColumn()->cloneEmpty(); - dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, getter, nested_size, 0, position_independent_encoding, path); - - /// Note: it's assumed that rows inserted into columnUnique get incremental indexes. - column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, dict_column->size()); - } + UInt64 num_keys; + readIntBinary(num_keys, *stream); + auto dict_column = column_with_dictionary.getUnique()->getNestedColumn()->cloneEmpty(); + dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, getter, num_keys, 0, position_independent_encoding, path); + indexes = column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, dict_column->size()); } path.back() = Substream::DictionaryIndexes; if (auto stream = getter(path)) - indexes_type->deserializeBinaryBulk(*column_with_dictionary.getIndexes(), *stream, limit, 0); + { + if (!indexes) + throw Exception("Dictionary keys wasn't deserialized", ErrorCodes::LOGICAL_ERROR); + + auto index_col = indexes_type->createColumn(); + indexes_type->deserializeBinaryBulk(*index_col, *stream, limit, 0); + column_with_dictionary.getIndexes()->insertRangeFrom(*index_col->index(indexes), 0, limit); + } } void DataTypeWithDictionary::serializeBinary(const Field & field, WriteBuffer & ostr) const diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index fe092268b42..22c5aad5636 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -92,7 +92,7 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy /// and name is encoded as a whole. stream_name += "%2E" + escapeForFileName(elem.tuple_element_name); } - else if (elem.type == Substream::DictionaryElements) + else if (elem.type == Substream::DictionaryKeys) stream_name += ".dict"; } return stream_name; diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index e6542c18b70..5fa04ae2ef1 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -80,7 +80,7 @@ public: TupleElement, - DictionaryElements, + DictionaryKeys, DictionaryIndexes, }; Type type; From 25f0f2f2d1965d5498026e247adb40564750bb8e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 21:54:58 +0300 Subject: [PATCH 053/315] added DataTypeWithDictionary serialization per granule --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 77164cdc522..724f40ec41d 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -78,8 +78,8 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( bool full_column = offset == 0 && limit >= indexes->size(); - auto unique_indexes = getUniqueIndex(full_column ? indexes : indexes->cut(offset, limit)); - auto used_keys = keys->index(*unique_indexes, 0); + ColumnPtr unique_indexes = getUniqueIndex(full_column ? indexes : indexes->cut(offset, limit)); + auto used_keys = keys->index(unique_indexes, 0); UInt64 used_keys_size = used_keys->size(); writeIntBinary(used_keys_size, *stream); @@ -122,7 +122,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( auto index_col = indexes_type->createColumn(); indexes_type->deserializeBinaryBulk(*index_col, *stream, limit, 0); - column_with_dictionary.getIndexes()->insertRangeFrom(*index_col->index(indexes), 0, limit); + column_with_dictionary.getIndexes()->insertRangeFrom(*index_col->index(indexes, 0), 0, limit); } } From 726780868e85c55f7f19323fdea7e2879c4b8071 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 21:56:28 +0300 Subject: [PATCH 054/315] added DataTypeWithDictionary serialization per granule --- dbms/src/Columns/ColumnsCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index 72ce9d0fcae..80a296de992 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -332,7 +332,7 @@ namespace detail MutableColumnPtr getUniqueIndexImpl(const PaddedPODArray & index) { HashSet hash_table; - for (auto val : index.getData()) + for (auto val : index) hash_table.insert(val); auto res_col = ColumnVector::create(); From afb1f6dd64a11543c85769e09f941fee80b1db9d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 22:11:41 +0300 Subject: [PATCH 055/315] added DataTypeWithDictionary serialization per granule --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 724f40ec41d..a5971f5916c 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -122,7 +122,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( auto index_col = indexes_type->createColumn(); indexes_type->deserializeBinaryBulk(*index_col, *stream, limit, 0); - column_with_dictionary.getIndexes()->insertRangeFrom(*index_col->index(indexes, 0), 0, limit); + column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(index_col, 0), 0, limit); } } From d13780d3ef05baf5f0e89aa25f03925fad190df4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 22:12:51 +0300 Subject: [PATCH 056/315] added DataTypeWithDictionary serialization per granule --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index a5971f5916c..c2f25349e3d 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -122,7 +122,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( auto index_col = indexes_type->createColumn(); indexes_type->deserializeBinaryBulk(*index_col, *stream, limit, 0); - column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(index_col, 0), 0, limit); + column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(std::move(index_col), 0), 0, limit); } } From 5e6d105fd5398ab8aacb2cb91e225eacb36b3e8c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 22:19:12 +0300 Subject: [PATCH 057/315] added DataTypeWithDictionary serialization per granule --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index c2f25349e3d..6b415da9442 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -83,7 +83,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( UInt64 used_keys_size = used_keys->size(); writeIntBinary(used_keys_size, *stream); - dictionary_type->serializeBinaryBulkWithMultipleStreams(*keys, getter, 0, 0, + dictionary_type->serializeBinaryBulkWithMultipleStreams(*used_keys, getter, 0, 0, position_independent_encoding, path); } path.back() = Substream::DictionaryIndexes; @@ -111,7 +111,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( readIntBinary(num_keys, *stream); auto dict_column = column_with_dictionary.getUnique()->getNestedColumn()->cloneEmpty(); dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, getter, num_keys, 0, position_independent_encoding, path); - indexes = column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, dict_column->size()); + indexes = column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, num_keys); } path.back() = Substream::DictionaryIndexes; From 5b0ac680ecdd7879efa08fc0721da05a5b8ed708 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 22:38:41 +0300 Subject: [PATCH 058/315] added DataTypeWithDictionary serialization per granule --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 6b415da9442..0b1e3cc29cf 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -72,13 +72,16 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( const auto & indexes = column_with_dictionary.getIndexesPtr(); const auto & keys = column_with_dictionary.getUnique()->getNestedColumn(); + if (limit == 0) + limit = indexes->size(); + path.push_back(Substream::DictionaryKeys); if (auto stream = getter(path)) { bool full_column = offset == 0 && limit >= indexes->size(); - ColumnPtr unique_indexes = getUniqueIndex(full_column ? indexes : indexes->cut(offset, limit)); + ColumnPtr unique_indexes = getUniqueIndex(full_column ? indexes : indexes->cut(offset, limit - offset)); auto used_keys = keys->index(unique_indexes, 0); UInt64 used_keys_size = used_keys->size(); From bfc42259cb5de499caf450c5a9c42bfe604c204a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 23:11:32 +0300 Subject: [PATCH 059/315] added DataTypeWithDictionary serialization per granule --- dbms/src/Columns/ColumnsCommon.cpp | 31 ++++++++++++++----- dbms/src/Columns/ColumnsCommon.h | 2 +- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 18 ++++++----- 3 files changed, 34 insertions(+), 17 deletions(-) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index 80a296de992..d84566361b6 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -323,30 +324,44 @@ namespace detail return nullptr; } + template + PaddedPODArray * getIndexesData(IColumn & indexes) + { + auto * column = typeid_cast *>(&indexes); + if (column) + return &column->getData(); + + return nullptr; + } + template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); template - MutableColumnPtr getUniqueIndexImpl(const PaddedPODArray & index) + MutableColumnPtr getUniqueIndexImpl(PaddedPODArray & index) { - HashSet hash_table; + HashMap hash_map; for (auto val : index) - hash_table.insert(val); + hash_map.insert({val, hash_map.size()}); auto res_col = ColumnVector::create(); auto & data = res_col->getData(); - data.reserve(hash_table.size()); - for (auto val : hash_table) - data.push_back(val); + data.resize(hash_map.size()); + for (auto val : hash_map) + data[val.second] = val.first; + + for (auto & ind : index) + ind = hash_map[ind]; return std::move(res_col); } } -MutableColumnPtr getUniqueIndex(const ColumnPtr & column) +/// Returns unique values of column. Write new index to column. +MutableColumnPtr makeSubIndex(IColumn & column) { if (auto * data_uint8 = detail::getIndexesData(column)) return detail::getUniqueIndexImpl(*data_uint8); @@ -357,7 +372,7 @@ MutableColumnPtr getUniqueIndex(const ColumnPtr & column) else if (auto * data_uint64 = detail::getIndexesData(column)) return detail::getUniqueIndexImpl(*data_uint64); else - throw Exception("Indexes column for getUniqueIndex must be ColumnUInt, got" + column->getName(), + throw Exception("Indexes column for makeSubindex must be ColumnUInt, got" + column->getName(), ErrorCodes::LOGICAL_ERROR); } diff --git a/dbms/src/Columns/ColumnsCommon.h b/dbms/src/Columns/ColumnsCommon.h index 06d3dcbd023..d2d4dc703f4 100644 --- a/dbms/src/Columns/ColumnsCommon.h +++ b/dbms/src/Columns/ColumnsCommon.h @@ -75,6 +75,6 @@ ColumnPtr selectIndexImpl(const Column & column, const ColumnPtr & indexes, size /// Get unique values from index column (ColumnUInt*). -MutableColumnPtr getUniqueIndex(const ColumnPtr & column); +MutableColumnPtr makeSubIndex(const ColumnPtr & column); } diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 0b1e3cc29cf..4f15537693d 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -69,8 +69,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( SubstreamPath path) const { const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); - const auto & indexes = column_with_dictionary.getIndexesPtr(); - const auto & keys = column_with_dictionary.getUnique()->getNestedColumn(); + MutableColumnPtr sub_index; if (limit == 0) limit = indexes->size(); @@ -78,10 +77,10 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( path.push_back(Substream::DictionaryKeys); if (auto stream = getter(path)) { - - bool full_column = offset == 0 && limit >= indexes->size(); - - ColumnPtr unique_indexes = getUniqueIndex(full_column ? indexes : indexes->cut(offset, limit - offset)); + const auto & indexes = column_with_dictionary.getIndexesPtr(); + const auto & keys = column_with_dictionary.getUnique()->getNestedColumn(); + sub_index = (*indexes->cut(offset, limit - offset)).mutate(); + ColumnPtr unique_indexes = makeSubIndex(sub_index); auto used_keys = keys->index(unique_indexes, 0); UInt64 used_keys_size = used_keys->size(); @@ -92,7 +91,10 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( path.back() = Substream::DictionaryIndexes; if (auto stream = getter(path)) { - indexes_type->serializeBinaryBulk(*indexes, *stream, offset, limit); + if (!sub_index) + throw Exception("Dictionary keys wasn't serialized", ErrorCodes::LOGICAL_ERROR); + + indexes_type->serializeBinaryBulk(*sub_index, *stream, offset, limit); } } @@ -112,7 +114,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( { UInt64 num_keys; readIntBinary(num_keys, *stream); - auto dict_column = column_with_dictionary.getUnique()->getNestedColumn()->cloneEmpty(); + auto dict_column = dictionary_type->cloneEmpty(); dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, getter, num_keys, 0, position_independent_encoding, path); indexes = column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, num_keys); } From 387227409adeffdecf3acdb6ed9f432143c082df Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 23:13:29 +0300 Subject: [PATCH 060/315] added DataTypeWithDictionary serialization per granule --- dbms/src/Columns/ColumnsCommon.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index d84566361b6..3eade6881ab 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -327,7 +327,7 @@ namespace detail template PaddedPODArray * getIndexesData(IColumn & indexes) { - auto * column = typeid_cast *>(&indexes); + auto * column = typeid_cast *>(&indexes); if (column) return &column->getData(); @@ -372,7 +372,7 @@ MutableColumnPtr makeSubIndex(IColumn & column) else if (auto * data_uint64 = detail::getIndexesData(column)) return detail::getUniqueIndexImpl(*data_uint64); else - throw Exception("Indexes column for makeSubindex must be ColumnUInt, got" + column->getName(), + throw Exception("Indexes column for makeSubindex must be ColumnUInt, got" + column.getName(), ErrorCodes::LOGICAL_ERROR); } From ad9cdee06a8a30566c9aa5f0093d387986e6961f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 23:14:52 +0300 Subject: [PATCH 061/315] added DataTypeWithDictionary serialization per granule --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 4f15537693d..93b7dcdf753 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -72,7 +72,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( MutableColumnPtr sub_index; if (limit == 0) - limit = indexes->size(); + limit = column.size(); path.push_back(Substream::DictionaryKeys); if (auto stream = getter(path)) @@ -114,7 +114,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( { UInt64 num_keys; readIntBinary(num_keys, *stream); - auto dict_column = dictionary_type->cloneEmpty(); + auto dict_column = dictionary_type->createColumn(); dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, getter, num_keys, 0, position_independent_encoding, path); indexes = column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, num_keys); } From 361d1193cb12c21998f529b23330e930b92b498e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 23:15:56 +0300 Subject: [PATCH 062/315] added DataTypeWithDictionary serialization per granule --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 93b7dcdf753..a9a81660fa5 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -80,7 +80,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( const auto & indexes = column_with_dictionary.getIndexesPtr(); const auto & keys = column_with_dictionary.getUnique()->getNestedColumn(); sub_index = (*indexes->cut(offset, limit - offset)).mutate(); - ColumnPtr unique_indexes = makeSubIndex(sub_index); + ColumnPtr unique_indexes = makeSubIndex(*sub_index); auto used_keys = keys->index(unique_indexes, 0); UInt64 used_keys_size = used_keys->size(); From 6fe65a4740c97f29f9c180d7997ea64d69c75a88 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 23:17:27 +0300 Subject: [PATCH 063/315] added DataTypeWithDictionary serialization per granule --- dbms/src/Columns/ColumnsCommon.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnsCommon.h b/dbms/src/Columns/ColumnsCommon.h index d2d4dc703f4..bc16af0b715 100644 --- a/dbms/src/Columns/ColumnsCommon.h +++ b/dbms/src/Columns/ColumnsCommon.h @@ -75,6 +75,6 @@ ColumnPtr selectIndexImpl(const Column & column, const ColumnPtr & indexes, size /// Get unique values from index column (ColumnUInt*). -MutableColumnPtr makeSubIndex(const ColumnPtr & column); +MutableColumnPtr makeSubIndex(IColumn & column); } From b45b2b9ea519caa55a7f9ea7ebcffd49c3303c95 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 May 2018 23:32:26 +0300 Subject: [PATCH 064/315] added DataTypeWithDictionary serialization per granule --- dbms/src/Functions/FunctionsConversion.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 7e1901f5a05..86cc7aefcda 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1208,6 +1208,7 @@ protected: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } private: @@ -1704,6 +1705,7 @@ protected: } bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } private: template From bd4d8a6766e4e5439621d12eb7069b1925763c79 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Thu, 17 May 2018 19:01:41 +0300 Subject: [PATCH 065/315] Add profile events and settings logging. [#CLICKHOUSE-2910] --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Common/ProfileEvents.cpp | 55 +++- dbms/src/Common/ProfileEvents.h | 9 + dbms/src/Common/TaskStatsInfoGetter.cpp | 243 ++++++++++++++++++ dbms/src/Common/TaskStatsInfoGetter.h | 32 +++ dbms/src/Common/ThreadStatus.cpp | 41 ++- dbms/src/Common/ThreadStatus.h | 12 +- dbms/src/Interpreters/DNSCacheUpdater.cpp | 2 +- dbms/src/Interpreters/ProcessList.cpp | 63 ++++- dbms/src/Interpreters/ProcessList.h | 58 +---- dbms/src/Interpreters/QueryLog.cpp | 39 +++ dbms/src/Interpreters/QueryLog.h | 4 + dbms/src/Interpreters/Settings.cpp | 39 +++ dbms/src/Interpreters/Settings.h | 7 + dbms/src/Interpreters/executeQuery.cpp | 31 ++- dbms/src/Interpreters/tests/CMakeLists.txt | 5 +- .../src/Interpreters/tests/internal_iotop.cpp | 168 ++++++++++++ .../System/StorageSystemProcesses.cpp | 64 ++--- debian/clickhouse-server.init | 2 + debian/clickhouse-server.service | 1 + 20 files changed, 763 insertions(+), 113 deletions(-) create mode 100644 dbms/src/Common/TaskStatsInfoGetter.cpp create mode 100644 dbms/src/Common/TaskStatsInfoGetter.h create mode 100644 dbms/src/Interpreters/tests/internal_iotop.cpp diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index c19279e9813..4dafa4104ad 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -373,6 +373,7 @@ namespace ErrorCodes extern const int TOO_MANY_ROWS_OR_BYTES = 396; extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW = 397; extern const int PTHREAD_ERROR = 398; + extern const int NETLINK_ERROR = 399; extern const int KEEPER_EXCEPTION = 999; diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 1882b907711..cdc09ea76b5 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -1,6 +1,7 @@ #include #include - +#include +#include /// Available events. Add something here as you wish. #define APPLY_FOR_EVENTS(M) \ @@ -143,12 +144,18 @@ M(RWLockReadersWaitMilliseconds) \ M(RWLockWritersWaitMilliseconds) \ M(NetworkErrors) \ + \ M(RealTimeMicroseconds) \ M(RusageUserTimeMicroseconds) \ M(RusageSystemTimeMicroseconds) \ M(RusagePageReclaims) \ M(RusagePageVoluntaryContextSwitches) \ - M(RusagePageInvoluntaryContextSwitches) + M(RusagePageInvoluntaryContextSwitches) \ + \ + M(OSReadBytes) \ + M(OSWriteBytes) \ + M(OSReadChars) \ + M(OSWriteChars) namespace ProfileEvents @@ -195,7 +202,6 @@ void Counters::getPartiallyAtomicSnapshot(Counters & res) const res.counters[i].store(counters[i], std::memory_order_relaxed); } - const char * getDescription(Event event) { static const char * descriptions[] = @@ -220,6 +226,49 @@ void increment(Event event, Count amount) global_counters.increment(event, amount); } +void Counters::dumpToArrayColumns(DB::IColumn * column_names_, DB::IColumn * column_values_, bool nonzero_only) +{ + /// Convert ptr and make simple check + auto column_names = (column_names_) ? &typeid_cast(*column_names_) : nullptr; + auto column_values = (column_values_) ? &typeid_cast(*column_values_) : nullptr; + + size_t size = 0; + + for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) + { + UInt64 value = counters[event].load(std::memory_order_relaxed); + + if (nonzero_only && 0 == value) + continue; + + ++size; + + if (column_names) + { + const char * desc = ProfileEvents::getDescription(event); + column_names->getData().insertData(desc, strlen(desc)); + } + + if (column_values) + column_values->getData().insert(value); + } + + if (column_names) + { + auto & offsets = column_names->getOffsets(); + offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size); + } + + /// Nested columns case + bool the_same_offsets = column_names && column_values && column_names->getOffsetsPtr() == column_values->getOffsetsPtr(); + + if (column_values && !the_same_offsets) + { + auto & offsets = column_values->getOffsets(); + offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size); + } +} + } #undef APPLY_FOR_EVENTS diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index 174c2945435..a87df2e57cf 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -4,6 +4,12 @@ #include #include +namespace DB +{ + +class IColumn; + +} /** Implements global counters for various events happening in the application * - for high level profiling. @@ -63,6 +69,9 @@ namespace ProfileEvents /// Reset metrics void resetCounters(); + /// Dumps profile events to two column Array(String) and Array(UInt64) + void dumpToArrayColumns(DB::IColumn * column_names, DB::IColumn * column_value, bool nonzero_only = true); + static const Event num_counters; }; diff --git a/dbms/src/Common/TaskStatsInfoGetter.cpp b/dbms/src/Common/TaskStatsInfoGetter.cpp new file mode 100644 index 00000000000..0973397fa83 --- /dev/null +++ b/dbms/src/Common/TaskStatsInfoGetter.cpp @@ -0,0 +1,243 @@ +#include "TaskStatsInfoGetter.h" +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +/// Based on: https://github.com/Tomas-M/iotop/tree/master/src + +/* + * Generic macros for dealing with netlink sockets. Might be duplicated + * elsewhere. It is recommended that commercial grade applications use + * libnl or libnetlink and use the interfaces provided by the library + */ +#define GENLMSG_DATA(glh) ((void *)((char*)NLMSG_DATA(glh) + GENL_HDRLEN)) +#define GENLMSG_PAYLOAD(glh) (NLMSG_PAYLOAD(glh, 0) - GENL_HDRLEN) +#define NLA_DATA(na) ((void *)((char*)(na) + NLA_HDRLEN)) +#define NLA_PAYLOAD(len) (len - NLA_HDRLEN) + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NETLINK_ERROR; +} + + +namespace +{ + +static size_t constexpr MAX_MSG_SIZE = 1024; + + +struct msgtemplate +{ + struct nlmsghdr n; + struct genlmsghdr g; + char buf[MAX_MSG_SIZE]; +}; + + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wsign-compare" + +int send_cmd(int sock_fd, __u16 nlmsg_type, __u32 nlmsg_pid, + __u8 genl_cmd, __u16 nla_type, + void *nla_data, int nla_len) +{ + struct nlattr *na; + struct sockaddr_nl nladdr; + int r, buflen; + char *buf; + + msgtemplate msg; + memset(&msg, 0, sizeof(msg)); + + msg.n.nlmsg_len = NLMSG_LENGTH(GENL_HDRLEN); + msg.n.nlmsg_type = nlmsg_type; + msg.n.nlmsg_flags = NLM_F_REQUEST; + msg.n.nlmsg_seq = 0; + msg.n.nlmsg_pid = nlmsg_pid; + msg.g.cmd = genl_cmd; + msg.g.version = 0x1; + + na = (struct nlattr *) GENLMSG_DATA(&msg); + na->nla_type = nla_type; + na->nla_len = nla_len + 1 + NLA_HDRLEN; + + memcpy(NLA_DATA(na), nla_data, nla_len); + msg.n.nlmsg_len += NLMSG_ALIGN(na->nla_len); + + buf = (char *) &msg; + buflen = msg.n.nlmsg_len ; + memset(&nladdr, 0, sizeof(nladdr)); + nladdr.nl_family = AF_NETLINK; + while ((r = sendto(sock_fd, buf, buflen, 0, (struct sockaddr *) &nladdr, + sizeof(nladdr))) < buflen) + { + if (r > 0) + { + buf += r; + buflen -= r; + } + else if (errno != EAGAIN) + return -1; + } + return 0; +} + + +int get_family_id(int nl_sock_fd) +{ + static char name[256]; + + struct + { + struct nlmsghdr n; + struct genlmsghdr g; + char buf[256]; + } ans; + + int id = 0; + struct nlattr *na; + int rep_len; + + strcpy(name, TASKSTATS_GENL_NAME); + if (send_cmd(nl_sock_fd, GENL_ID_CTRL, getpid(), CTRL_CMD_GETFAMILY, + CTRL_ATTR_FAMILY_NAME, (void *) name, + strlen(TASKSTATS_GENL_NAME) + 1)) + return 0; + + rep_len = recv(nl_sock_fd, &ans, sizeof(ans), 0); + if (ans.n.nlmsg_type == NLMSG_ERROR + || (rep_len < 0) || !NLMSG_OK((&ans.n), rep_len)) + return 0; + + na = (struct nlattr *) GENLMSG_DATA(&ans); + na = (struct nlattr *) ((char *) na + NLA_ALIGN(na->nla_len)); + if (na->nla_type == CTRL_ATTR_FAMILY_ID) + id = *(__u16 *) NLA_DATA(na); + + return id; +} + +bool get_taskstats(int nl_sock_fd, int nl_family_id, pid_t xxxid, struct taskstats & out_stats, Exception * out_exception = nullptr) +{ + if (send_cmd(nl_sock_fd, nl_family_id, xxxid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &xxxid, sizeof(pid_t))) + throwFromErrno("Can't send a Netlink command"); + + msgtemplate msg; + int rv = recv(nl_sock_fd, &msg, sizeof(msg), 0); + + if (msg.n.nlmsg_type == NLMSG_ERROR || !NLMSG_OK((&msg.n), rv)) + { + struct nlmsgerr *err = static_cast(NLMSG_DATA(&msg)); + Exception e("Can't get Netlink response, error=" + std::to_string(err->error), ErrorCodes::NETLINK_ERROR); + + if (out_exception) + { + *out_exception = std::move(e); + return false; + } + + throw Exception(std::move(e)); + } + + rv = GENLMSG_PAYLOAD(&msg.n); + + struct nlattr *na = (struct nlattr *) GENLMSG_DATA(&msg); + int len = 0; + + while (len < rv) + { + len += NLA_ALIGN(na->nla_len); + + if (na->nla_type == TASKSTATS_TYPE_AGGR_TGID + || na->nla_type == TASKSTATS_TYPE_AGGR_PID) + { + int aggr_len = NLA_PAYLOAD(na->nla_len); + int len2 = 0; + + na = (struct nlattr *) NLA_DATA(na); + while (len2 < aggr_len) + { + if (na->nla_type == TASKSTATS_TYPE_STATS) + { + struct taskstats *ts = static_cast(NLA_DATA(na)); + out_stats = *ts; + } + len2 += NLA_ALIGN(na->nla_len); + na = (struct nlattr *) ((char *) na + len2); + } + } + na = (struct nlattr *) ((char *) GENLMSG_DATA(&msg) + len); + } + + return true; +} + +#pragma GCC diagnostic pop + +} + + +TaskStatsInfoGetter::TaskStatsInfoGetter() +{ + netlink_socket_fd = socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC); + if (netlink_socket_fd < 0) + throwFromErrno("Can't create PF_NETLINK socket"); + + struct sockaddr_nl addr; + memset(&addr, 0, sizeof(addr)); + addr.nl_family = AF_NETLINK; + + if (bind(netlink_socket_fd, (struct sockaddr *) &addr, sizeof(addr)) < 0) + throwFromErrno("Can't bind PF_NETLINK socket"); + + netlink_family_id = get_family_id(netlink_socket_fd); +} + +void TaskStatsInfoGetter::getStat(::taskstats & stat, int tid) const +{ + if (tid < 0) + tid = getCurrentTID(); + + get_taskstats(netlink_socket_fd, netlink_family_id, tid, stat); +} + +bool TaskStatsInfoGetter::tryGetStat(::taskstats & stat, int tid) const +{ + if (tid < 0) + tid = getCurrentTID(); + + Exception e; + return get_taskstats(netlink_socket_fd, netlink_family_id, tid, stat, &e); +} + +TaskStatsInfoGetter::~TaskStatsInfoGetter() +{ + if (netlink_socket_fd > -1) + close(netlink_socket_fd); +} + +int TaskStatsInfoGetter::getCurrentTID() +{ + return static_cast(syscall(SYS_gettid)); +} + + +} diff --git a/dbms/src/Common/TaskStatsInfoGetter.h b/dbms/src/Common/TaskStatsInfoGetter.h new file mode 100644 index 00000000000..1d77b2fbb20 --- /dev/null +++ b/dbms/src/Common/TaskStatsInfoGetter.h @@ -0,0 +1,32 @@ +#pragma once +#include + +struct taskstats; + + +namespace DB +{ + +/// Get taskstat infor from OS kernel via Netlink protocol +class TaskStatsInfoGetter +{ +public: + + TaskStatsInfoGetter(); + TaskStatsInfoGetter(const TaskStatsInfoGetter &) = delete; + + void getStat(::taskstats & stat, int tid = -1) const; + bool tryGetStat(::taskstats & stat, int tid = -1) const; + + /// Returns Linux internal thread id + static int getCurrentTID(); + + ~TaskStatsInfoGetter(); + +private: + + int netlink_socket_fd = -1; + int netlink_family_id = 0; +}; + +} diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index b52099845b0..a1159897483 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -2,12 +2,14 @@ #include #include #include +#include +#include + #include #include #include - -#include +#include namespace ProfileEvents @@ -18,6 +20,11 @@ namespace ProfileEvents extern const Event RusagePageReclaims; extern const Event RusagePageVoluntaryContextSwitches; extern const Event RusagePageInvoluntaryContextSwitches; + + extern const Event OSReadBytes; + extern const Event OSWriteBytes; + extern const Event OSReadChars; + extern const Event OSWriteChars; } @@ -51,7 +58,7 @@ struct RusageCounters UInt64 involuntary_context_switches = 0; RusageCounters() = default; - RusageCounters(const struct rusage & rusage, UInt64 real_time_) + RusageCounters(const ::rusage & rusage, UInt64 real_time_) { set(rusage, real_time_); } @@ -66,18 +73,18 @@ struct RusageCounters static RusageCounters current() { RusageCounters res; - res.setFromCurrent(); + res.setCurrent(); return res; } - void setFromCurrent() + void setCurrent() { - struct rusage rusage; + rusage rusage; getrusage(RUSAGE_THREAD, &rusage); set(rusage, getCurrentTimeMicroseconds()); } - void set(const struct rusage & rusage, UInt64 real_time_) + void set(const ::rusage & rusage, UInt64 real_time_) { real_time = real_time_; user_time = rusage.ru_utime.tv_sec * 1000000UL + rusage.ru_utime.tv_usec; @@ -110,6 +117,7 @@ struct RusageCounters struct ThreadStatus::Impl { RusageCounters last_rusage; + TaskStatsInfoGetter info_getter; }; @@ -140,6 +148,7 @@ static void thread_create_at_exit_key() { ThreadStatus::ThreadStatus() : poco_thread_number(Poco::ThreadNumber::get()), performance_counters(ProfileEvents::Level::Thread), + os_thread_id(TaskStatsInfoGetter::getCurrentTID()), log(&Poco::Logger::get("ThreadStatus")) { impl = std::make_shared(); @@ -198,13 +207,29 @@ void ThreadStatus::init(QueryStatus * parent_query_, ProfileEvents::Counters * p void ThreadStatus::onStart() { /// First init of thread rusage counters, set real time to zero, other metrics remain as is - impl->last_rusage.setFromCurrent(); + impl->last_rusage.setCurrent(); RusageCounters::incrementProfileEvents(impl->last_rusage, RusageCounters::zeros(impl->last_rusage.real_time)); } void ThreadStatus::onExit() { RusageCounters::updateProfileEvents(impl->last_rusage); + + try + { + ::taskstats stat; + TaskStatsInfoGetter info_getter; + info_getter.getStat(stat); + + ProfileEvents::increment(ProfileEvents::OSReadBytes, stat.read_bytes); + ProfileEvents::increment(ProfileEvents::OSWriteBytes, stat.write_bytes); + ProfileEvents::increment(ProfileEvents::OSReadChars, stat.read_char); + ProfileEvents::increment(ProfileEvents::OSWriteChars, stat.write_char); + } + catch (...) + { + tryLogCurrentException(log); + } } void ThreadStatus::reset() diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index ee0d2072b9c..cb45c11f280 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -15,19 +15,21 @@ namespace Poco namespace DB { -struct QueryStatus; -struct ThreadStatus; +class QueryStatus; +class ThreadStatus; struct ScopeCurrentThread; using ThreadStatusPtr = std::shared_ptr; -struct ThreadStatus : public ext::shared_ptr_helper +class ThreadStatus : public ext::shared_ptr_helper { +public: + UInt32 poco_thread_number = 0; QueryStatus * parent_query = nullptr; ProfileEvents::Counters performance_counters; MemoryTracker memory_tracker; - bool thread_exited = false; + int os_thread_id = -1; std::mutex mutex; void init(QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker); @@ -48,13 +50,13 @@ struct ThreadStatus : public ext::shared_ptr_helper ThreadStatus(); bool initialized = false; + bool thread_exited = false; Poco::Logger * log; struct Impl; std::shared_ptr impl; }; - extern thread_local ThreadStatusPtr current_thread; } diff --git a/dbms/src/Interpreters/DNSCacheUpdater.cpp b/dbms/src/Interpreters/DNSCacheUpdater.cpp index 16653dff3c8..6c2d1d8ee98 100644 --- a/dbms/src/Interpreters/DNSCacheUpdater.cpp +++ b/dbms/src/Interpreters/DNSCacheUpdater.cpp @@ -64,7 +64,7 @@ DNSCacheUpdater::DNSCacheUpdater(Context & context_) bool DNSCacheUpdater::run() { /// TODO: Ensusre that we get global counter (not thread local) - auto num_current_network_exceptions = ProfileEvents::counters[ProfileEvents::NetworkErrors].load(std::memory_order_relaxed); + auto num_current_network_exceptions = ProfileEvents::global_counters[ProfileEvents::NetworkErrors].load(std::memory_order_relaxed); if (num_current_network_exceptions >= last_num_network_erros + min_errors_to_update_cache && time(nullptr) > last_update_time + min_update_period_seconds) diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 7611f33d9b3..5f5f0242bcd 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -1,13 +1,14 @@ #include #include +#include #include #include #include #include +#include #include #include #include -#include #include #include #include @@ -73,11 +74,13 @@ static bool isUnlimitedQuery(const IAST * ast) } -ProcessList::EntryPtr ProcessList::insert( - const String & query_, const IAST * ast, const ClientInfo & client_info, const Settings & settings) +ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * ast, Context & query_context) { EntryPtr res; + const ClientInfo & client_info = query_context.getClientInfo(); + const Settings & settings = query_context.getSettingsRef(); + if (client_info.current_query_id.empty()) throw Exception("Query id cannot be empty", ErrorCodes::LOGICAL_ERROR); @@ -135,6 +138,8 @@ ProcessList::EntryPtr ProcessList::insert( res = std::make_shared(*this, process_it); + process_it->query_context = &query_context; + if (!client_info.current_query_id.empty()) { ProcessListForUser & user_process_list = user_to_queries[client_info.current_user]; @@ -394,6 +399,58 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr } +QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const +{ + QueryStatusInfo res; + + res.query = query; + res.client_info = client_info; + res.elapsed_seconds = watch.elapsedSeconds(); + res.is_cancelled = is_killed.load(std::memory_order_relaxed); + res.read_rows = progress_in.rows; + res.read_bytes = progress_in.bytes; + res.total_rows = progress_in.total_rows; + res.written_rows = progress_out.rows; + res.written_bytes = progress_out.bytes; + res.memory_usage = memory_tracker.get(); + res.peak_memory_usage = memory_tracker.getPeak(); + + if (get_thread_list) + { + std::lock_guard lock(threads_mutex); + res.thread_numbers.reserve(thread_statuses.size()); + + for (auto & thread_status_elem : thread_statuses) + res.thread_numbers.emplace_back(thread_status_elem.second->poco_thread_number); + } + + if (get_profile_events) + { + res.profile_counters = std::make_shared(ProfileEvents::Level::Process); + performance_counters.getPartiallyAtomicSnapshot(*res.profile_counters); + } + + if (get_settings && query_context) + res.query_settings = std::make_shared(query_context->getSettingsRef()); + + return res; +} + + +ProcessList::Info ProcessList::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const +{ + Info per_query_infos; + + std::lock_guard lock(mutex); + + per_query_infos.reserve(processes.size()); + for (const auto & process : processes) + per_query_infos.emplace_back(process.getInfo(get_thread_list, get_profile_events, get_settings)); + + return per_query_infos; +} + + ProcessListForUser::ProcessListForUser() : user_performance_counters(ProfileEvents::Level::User, &ProfileEvents::global_counters) {} diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index 2cd52b8c481..ac0958a9079 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -34,6 +34,7 @@ namespace DB class IStorage; using StoragePtr = std::shared_ptr; using Tables = std::map; +class Context; struct Settings; class IAST; @@ -66,7 +67,8 @@ struct QueryStatusInfo /// Optional fields, filled by request std::vector thread_numbers; - std::unique_ptr profile_counters; + std::shared_ptr profile_counters; + std::shared_ptr query_settings; }; @@ -80,6 +82,7 @@ protected: String query; ClientInfo client_info; + Context * query_context = nullptr; Stopwatch watch; @@ -172,25 +175,9 @@ public: return !is_killed.load(std::memory_order_relaxed); } + QueryStatusInfo getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const; - QueryStatusInfo getInfo() const - { - QueryStatusInfo res; - - res.query = query; - res.client_info = client_info; - res.elapsed_seconds = watch.elapsedSeconds(); - res.is_cancelled = is_killed.load(std::memory_order_relaxed); - res.read_rows = progress_in.rows; - res.read_bytes = progress_in.bytes; - res.total_rows = progress_in.total_rows; - res.written_rows = progress_out.rows; - res.written_bytes = progress_out.bytes; - res.memory_usage = memory_tracker.get(); - res.peak_memory_usage = memory_tracker.getPeak(); - - return res; - } + const Context * tryGetQueryContext() const { return query_context; } /// Copies pointers to in/out streams void setQueryStreams(const BlockIO & io); @@ -308,42 +295,13 @@ public: * If timeout is passed - throw an exception. * Don't count KILL QUERY queries. */ - EntryPtr insert(const String & query_, const IAST * ast, const ClientInfo & client_info, const Settings & settings); + EntryPtr insert(const String & query_, const IAST * ast, Context & query_context); /// Number of currently executing queries. size_t size() const { return processes.size(); } /// Get current state of process list. - Info getInfo(bool get_thread_list = false, bool get_profile_events = false) const - { - Info per_query_infos; - - std::lock_guard lock(mutex); - - per_query_infos.reserve(processes.size()); - for (const auto & process : processes) - { - per_query_infos.emplace_back(process.getInfo()); - QueryStatusInfo & current_info = per_query_infos.back(); - - if (get_thread_list) - { - std::lock_guard lock(process.threads_mutex); - current_info.thread_numbers.reserve(process.thread_statuses.size()); - - for (auto & thread_status_elem : process.thread_statuses) - current_info.thread_numbers.emplace_back(thread_status_elem.second->poco_thread_number); - } - - if (get_profile_events) - { - current_info.profile_counters = std::make_unique(ProfileEvents::Level::Process); - process.performance_counters.getPartiallyAtomicSnapshot(*current_info.profile_counters); - } - } - - return per_query_infos; - } + Info getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const; void setMaxSize(size_t max_size_) { diff --git a/dbms/src/Interpreters/QueryLog.cpp b/dbms/src/Interpreters/QueryLog.cpp index fb197d85e00..8385843fcdf 100644 --- a/dbms/src/Interpreters/QueryLog.cpp +++ b/dbms/src/Interpreters/QueryLog.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -66,6 +67,12 @@ Block QueryLogElement::createBlock() {ColumnString::create(), std::make_shared(), "quota_key"}, {ColumnUInt32::create(), std::make_shared(), "revision"}, + + { std::make_shared(std::make_shared()), "thread_numbers" }, + { std::make_shared(std::make_shared()), "ProfileEvents.Names" }, + { std::make_shared(std::make_shared()), "ProfileEvents.Values" }, + { std::make_shared(std::make_shared()), "Settings.Names" }, + { std::make_shared(std::make_shared()), "Settings.Values" } }; } @@ -146,6 +153,38 @@ void QueryLogElement::appendToBlock(Block & block) const columns[i++]->insert(UInt64(ClickHouseRevision::get())); + { + Array threads_array; + threads_array.reserve(thread_numbers.size()); + for (const UInt32 thread_number : thread_numbers) + threads_array.emplace_back(UInt64(thread_number)); + columns[i++]->insert(threads_array); + } + + if (profile_counters) + { + auto column_names = columns[i++].get(); + auto column_values = columns[i++].get(); + profile_counters->dumpToArrayColumns(column_names, column_values, true); + } + else + { + columns[i++]->insertDefault(); + columns[i++]->insertDefault(); + } + + if (query_settings) + { + auto column_names = columns[i++].get(); + auto column_values = columns[i++].get(); + query_settings->dumpToArrayColumns(column_names, column_values, true); + } + else + { + columns[i++]->insertDefault(); + columns[i++]->insertDefault(); + } + block.setColumns(std::move(columns)); } diff --git a/dbms/src/Interpreters/QueryLog.h b/dbms/src/Interpreters/QueryLog.h index 8b4c7734ebe..5f4a09e6736 100644 --- a/dbms/src/Interpreters/QueryLog.h +++ b/dbms/src/Interpreters/QueryLog.h @@ -53,6 +53,10 @@ struct QueryLogElement ClientInfo client_info; + std::vector thread_numbers; + std::shared_ptr profile_counters; + std::shared_ptr query_settings; + static std::string name() { return "QueryLog"; } static Block createBlock(); diff --git a/dbms/src/Interpreters/Settings.cpp b/dbms/src/Interpreters/Settings.cpp index e6a8544be1b..2ab2e4db0ce 100644 --- a/dbms/src/Interpreters/Settings.cpp +++ b/dbms/src/Interpreters/Settings.cpp @@ -1,4 +1,7 @@ #include +#include +#include +#include namespace DB @@ -172,4 +175,40 @@ void Settings::serialize(WriteBuffer & buf) const #undef WRITE } +void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_values_, bool changed_only) +{ + /// Convert ptr and make simple check + auto column_names = (column_names_) ? &typeid_cast(*column_names_) : nullptr; + auto column_values = (column_values_) ? &typeid_cast(*column_values_) : nullptr; + + size_t size = 0; + +#define ADD_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) \ + if (!changed_only || NAME.changed) \ + { \ + if (column_names) \ + column_names->getData().insertData(#NAME, strlen(#NAME)); \ + if (column_values) \ + column_values->getData().insert(NAME.toString()); \ + ++size; \ + } + APPLY_FOR_SETTINGS(ADD_SETTING) +#undef ADD_SETTING + + if (column_names) + { + auto & offsets = column_names->getOffsets(); + offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size); + } + + /// Nested columns case + bool the_same_offsets = column_names && column_values && column_names->getOffsetsPtr() == column_values->getOffsetsPtr(); + + if (column_values && !the_same_offsets) + { + auto & offsets = column_values->getOffsets(); + offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size); + } +} + } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index ff275938ad2..2a80e4da047 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -10,6 +10,8 @@ namespace DB { +class IColumn; + /** Settings of query execution. */ struct Settings @@ -253,6 +255,8 @@ struct Settings M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.") \ M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.") \ M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \ + M(SettingBool, log_profile_events, true, "Log query settings into the query_log.") \ + M(SettingBool, log_query_settings, true, "Log query performance statistics into the query_log.") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; @@ -292,6 +296,9 @@ struct Settings /// Write changed settings to buffer. (For example, to be sent to remote server.) void serialize(WriteBuffer & buf) const; + + /// Dumps profile events to two column Array(String) and Array(UInt64) + void dumpToArrayColumns(IColumn * column_names, IColumn * column_values, bool changed_only = true); }; diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index bfa01d670d8..587bd6498eb 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -198,12 +198,7 @@ static std::tuple executeQueryImpl( ProcessList::EntryPtr process_list_entry; if (!internal && nullptr == typeid_cast(&*ast)) { - process_list_entry = context.getProcessList().insert( - query, - ast.get(), - context.getClientInfo(), - settings); - + process_list_entry = context.getProcessList().insert(query, ast.get(), context); context.setProcessListElement(&process_list_entry->get()); } @@ -264,6 +259,9 @@ static std::tuple executeQueryImpl( /// Log into system table start of query execution, if need. if (log_queries) { + if (settings.log_query_settings) + elem.query_settings = std::make_shared(context.getSettingsRef()); + if (auto query_log = context.getQueryLog()) query_log->add(elem); } @@ -272,11 +270,12 @@ static std::tuple executeQueryImpl( res.finish_callback = [elem, &context, log_queries] (IBlockInputStream * stream_in, IBlockOutputStream * stream_out) mutable { QueryStatus * process_list_elem = context.getProcessListElement(); + const Settings & settings = context.getSettingsRef(); if (!process_list_elem) return; - ProcessInfo info = process_list_elem->getInfo(); + QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events); double elapsed_seconds = info.elapsed_seconds; @@ -297,18 +296,18 @@ static std::tuple executeQueryImpl( { if (auto profiling_stream = dynamic_cast(stream_in)) { - const BlockStreamProfileInfo & info = profiling_stream->getProfileInfo(); + const BlockStreamProfileInfo & stream_in_info = profiling_stream->getProfileInfo(); /// NOTE: INSERT SELECT query contains zero metrics - elem.result_rows = info.rows; - elem.result_bytes = info.bytes; + elem.result_rows = stream_in_info.rows; + elem.result_bytes = stream_in_info.bytes; } } else if (stream_out) /// will be used only for ordinary INSERT queries { if (auto counting_stream = dynamic_cast(stream_out)) { - /// NOTE: Redundancy. The same values could be extracted from process_list_elem->progress_out. + /// NOTE: Redundancy. The same values coulld be extracted from process_list_elem->progress_out.query_settings = process_list_elem->progress_in elem.result_rows = counting_stream->getProgress().rows; elem.result_bytes = counting_stream->getProgress().bytes; } @@ -323,6 +322,9 @@ static std::tuple executeQueryImpl( << formatReadableSizeWithBinarySuffix(elem.read_bytes / elapsed_seconds) << "/sec."); } + elem.thread_numbers = std::move(info.thread_numbers); + elem.profile_counters = std::move(info.profile_counters); + if (log_queries) { if (auto query_log = context.getQueryLog()) @@ -341,10 +343,11 @@ static std::tuple executeQueryImpl( elem.exception = getCurrentExceptionMessage(false); QueryStatus * process_list_elem = context.getProcessListElement(); + const Settings & settings = context.getSettingsRef(); if (process_list_elem) { - ProcessInfo info = process_list_elem->getInfo(); + QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events); elem.query_duration_ms = info.elapsed_seconds * 1000; @@ -352,11 +355,15 @@ static std::tuple executeQueryImpl( elem.read_bytes = info.read_bytes; elem.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0; + + elem.thread_numbers = std::move(info.thread_numbers); + elem.profile_counters = std::move(info.profile_counters); } setExceptionStackTrace(elem); logException(context, elem); + /// In case of exception we log internal queries also if (log_queries) { if (auto query_log = context.getQueryLog()) diff --git a/dbms/src/Interpreters/tests/CMakeLists.txt b/dbms/src/Interpreters/tests/CMakeLists.txt index ad3a24d8608..3d2e38c1512 100644 --- a/dbms/src/Interpreters/tests/CMakeLists.txt +++ b/dbms/src/Interpreters/tests/CMakeLists.txt @@ -48,4 +48,7 @@ target_link_libraries (in_join_subqueries_preprocessor dbms) add_check(in_join_subqueries_preprocessor) add_executable (users users.cpp) -target_link_libraries (users dbms ${Boost_FILESYSTEM_LIBRARY} dbms) +target_link_libraries (users dbms ${Boost_FILESYSTEM_LIBRARY}) + +add_executable (internal_iotop internal_iotop.cpp) +target_link_libraries (internal_iotop dbms) diff --git a/dbms/src/Interpreters/tests/internal_iotop.cpp b/dbms/src/Interpreters/tests/internal_iotop.cpp new file mode 100644 index 00000000000..d8c5a5c6545 --- /dev/null +++ b/dbms/src/Interpreters/tests/internal_iotop.cpp @@ -0,0 +1,168 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + + + +std::mutex mutex; + + +std::ostream & operator << (std::ostream & stream, const ::taskstats & stat) +{ +#define PRINT(field) (stream << #field << " " << stat.field) + + PRINT(ac_pid) << ", "; + + PRINT(read_bytes) << ", "; + PRINT(write_bytes) << ", "; + + PRINT(read_char) << ", "; + PRINT(write_char) << ", "; + + PRINT(swapin_delay_total) << ", "; + PRINT(blkio_delay_total) << ", "; + PRINT(cpu_delay_total) << ", "; + + PRINT(ac_pid) << ", "; + + PRINT(ac_utime) << ", "; + PRINT(ac_stime) << ", "; + +#undef PRINT + + return stream; +} + +using namespace DB; + + +void do_io(size_t id) +{ + ::taskstats stat; + int tid = TaskStatsInfoGetter::getCurrentTID(); + TaskStatsInfoGetter get_info; + + if (!get_info.tryGetStat(stat, tid)) + { + std::lock_guard lock(mutex); + std::cerr << "#" << id << ", tid " << tid << ". Can't get stat\n"; + } + else + { + std::lock_guard lock(mutex); + std::cerr << "#" << id << ", tid " << tid << ", intitial\n" << stat << "\n"; + } + + size_t copy_size = DBMS_DEFAULT_BUFFER_SIZE * (1 + id); + std::string path_dst = "test_out_" + std::to_string(id); + + { + ReadBufferFromFile rb("/dev/urandom"); + WriteBufferFromFile wb(path_dst, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC | O_DIRECT); + copyData(rb, wb, copy_size); + } + + get_info.getStat(stat, tid); + { + std::lock_guard lock(mutex); + std::cerr << "#" << id << ", tid " << tid << ", step1\n" << stat << "\n"; + } + + { + ReadBufferFromFile rb(path_dst); + WriteBufferFromOwnString wb; + copyData(rb, wb, copy_size); + } + + get_info.getStat(stat, tid); + { + std::lock_guard lock(mutex); + std::cerr << "#" << id << ", tid " << tid << ", step2\n" << stat << "\n"; + } + + { + ReadBufferFromFile rb(path_dst); + WriteBufferFromOwnString wb; + copyData(rb, wb, copy_size); + } + + get_info.getStat(stat, tid); + { + std::lock_guard lock(mutex); + std::cerr << "#" << id << ", tid " << tid << ", step2\n" << stat << "\n"; + } + + Poco::File(path_dst).remove(false); +} + +void test_perf() +{ + + ::taskstats stat; + int tid = TaskStatsInfoGetter::getCurrentTID(); + TaskStatsInfoGetter get_info; + + rusage rusage; + timespec ts; + + constexpr size_t num_samples = 1000000; + { + Stopwatch watch; + for (size_t i = 0; i < num_samples; ++i) + { + getrusage(RUSAGE_THREAD, &rusage); + clock_gettime(CLOCK_MONOTONIC, &ts); + } + + auto ms = watch.elapsedMilliseconds(); + if (ms > 0) + std::cerr << "Rusage: " << double(ms) / num_samples << " ms per call, " << 1000 * num_samples / ms << " calls per second\n"; + } + + { + Stopwatch watch; + for (size_t i = 0; i < num_samples; ++i) + get_info.getStat(stat, tid); + + auto ms = watch.elapsedMilliseconds(); + if (ms > 0) + std::cerr << "Netlink: " << double(ms) / num_samples << " ms per call, " << 1000 * num_samples / ms << " calls per second\n"; + } + + std::cerr << stat << "\n"; +} + +int main() +try +{ + std::cerr << "pid " << getpid() << "\n"; + + size_t num_threads = 2; + ThreadPool pool(num_threads); + for (size_t i = 0; i < num_threads; ++i) + pool.schedule([i]() { do_io(i); }); + + pool.wait(); + + + test_perf(); + + + return 0; +} +catch (...) +{ + std::cerr << getCurrentExceptionMessage(true); + return -1; +} + diff --git a/dbms/src/Storages/System/StorageSystemProcesses.cpp b/dbms/src/Storages/System/StorageSystemProcesses.cpp index 3bb5d658f95..9a9f39208bf 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.cpp +++ b/dbms/src/Storages/System/StorageSystemProcesses.cpp @@ -7,7 +7,11 @@ #include #include #include +#include #include +#include +#include +#include namespace DB @@ -57,15 +61,11 @@ StorageSystemProcesses::StorageSystemProcesses(const std::string & name_) })); virtual_columns = ColumnsWithTypeAndName{ - { - std::make_shared(std::make_shared()), - "thread_numbers" - }, - { - std::make_shared(std::make_shared( - DataTypes{std::make_shared(), std::make_shared()})), - "profile_counters" - } + { std::make_shared(std::make_shared()), "thread_numbers" }, + { std::make_shared(std::make_shared()), "ProfileEvents.Names" }, + { std::make_shared(std::make_shared()), "ProfileEvents.Values" }, + { std::make_shared(std::make_shared()), "Settings.Names" }, + { std::make_shared(std::make_shared()), "Settings.Values" } }; } @@ -81,15 +81,18 @@ BlockInputStreams StorageSystemProcesses::read( processed_stage = QueryProcessingStage::FetchColumns; auto virtual_columns_processor = getVirtualColumnsProcessor(); - bool has_thread_numbers, has_profile_counters; - Names real_columns = virtual_columns_processor.process(column_names, {&has_thread_numbers, &has_profile_counters}); + bool has_thread_numbers, has_profile_events_names, has_profile_events_values, has_settigns_names, has_settings_values; + std::vector flags{&has_thread_numbers, &has_profile_events_names, &has_profile_events_values, &has_settigns_names, &has_settings_values}; + + Names real_columns = virtual_columns_processor.process(column_names, flags); check(real_columns); Block res_block = getSampleBlock().cloneEmpty(); virtual_columns_processor.appendVirtualColumns(res_block); MutableColumns res_columns = res_block.cloneEmptyColumns(); - ProcessList::Info info = context.getProcessList().getInfo(has_thread_numbers, has_profile_counters); + ProcessList::Info info = context.getProcessList().getInfo(has_thread_numbers, has_profile_events_names || has_profile_events_values, + has_settigns_names || has_settings_values); for (const auto & process : info) { @@ -126,31 +129,32 @@ BlockInputStreams StorageSystemProcesses::read( if (has_thread_numbers) { - Array thread_numbers; - thread_numbers.reserve(process.thread_numbers.size()); - + Array threads_array; + threads_array.reserve(process.thread_numbers.size()); for (const UInt32 thread_number : process.thread_numbers) - thread_numbers.emplace_back(UInt64(thread_number)); - - res_columns[i++]->insert(std::move(thread_numbers)); + threads_array.emplace_back(UInt64(thread_number)); + res_columns[i++]->insert(threads_array); } - if (has_profile_counters) + if (has_profile_events_names || has_profile_events_values) { - Array profile_counters; - profile_counters.reserve(ProfileEvents::Counters::num_counters); + IColumn * column_names = has_profile_events_names ? res_columns[i++].get() : nullptr; + IColumn * column_values = has_profile_events_values ? res_columns[i++].get() : nullptr; + process.profile_counters->dumpToArrayColumns(column_names, column_values, true); + } - for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) + if (has_settigns_names || has_settings_values) + { + IColumn * column_names = has_settigns_names ? res_columns[i++].get() : nullptr; + IColumn * column_values = has_settings_values ? res_columns[i++].get() : nullptr; + + if (process.query_settings) + process.query_settings->dumpToArrayColumns(column_names, column_values, true); + else { - Array name_and_counter{ - String(ProfileEvents::getDescription(event)), - UInt64((*process.profile_counters)[event].load(std::memory_order_relaxed)) - }; - - profile_counters.emplace_back(Tuple(std::move(name_and_counter))); + column_names->insertDefault(); + column_values->insertDefault(); } - - res_columns[i++]->insert(std::move(profile_counters)); } } diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index 8d0a75f573e..719b7c42c44 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -161,6 +161,8 @@ start() else mkdir -p $CLICKHOUSE_PIDDIR chown -R $CLICKHOUSE_USER:$CLICKHOUSE_GROUP $CLICKHOUSE_PIDDIR + # Set net_admin capabilities to support ClickHouse better introspection + su -s setcap cap_net_admin=+ep "$BINDIR/$PROGRAM" initdb if ! is_running; then # Lock should not be held while running child process, so we release the lock. Note: obviously, there is race condition. diff --git a/debian/clickhouse-server.service b/debian/clickhouse-server.service index 64d2cbacf6e..4efb26352ba 100644 --- a/debian/clickhouse-server.service +++ b/debian/clickhouse-server.service @@ -10,6 +10,7 @@ RestartSec=30 ExecStart=/usr/bin/clickhouse-server --config=/etc/clickhouse-server/config.xml LimitCORE=infinity LimitNOFILE=500000 +CapabilityBoundingSet=CAP_NET_ADMIN [Install] WantedBy=multi-user.target From 559d944412dfa52226142b6c0eb2ac503e06fa0a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 May 2018 14:52:02 +0300 Subject: [PATCH 066/315] Fixed build. --- dbms/src/Functions/FunctionsMiscellaneous.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index dfbd20b8763..97c11e8dcb8 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -1850,7 +1850,7 @@ public: return std::make_shared(arguments[0], std::make_shared()); } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { auto arg_num = arguments[0]; const auto & arg = block.getByPosition(arg_num); @@ -1885,7 +1885,7 @@ public: return type->getIndexesType(); } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { auto arg_num = arguments[0]; const auto & arg = block.getByPosition(arg_num); @@ -1918,7 +1918,7 @@ public: return type->getDictionaryType(); } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { auto arg_num = arguments[0]; const auto & arg = block.getByPosition(arg_num); From d81744fd0631471f0f6f876e913f886345deac82 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Mon, 28 May 2018 22:53:03 +0300 Subject: [PATCH 067/315] Add more perfomance counters. [#CLICKHOUSE-2910] --- dbms/src/Common/BackgroundSchedulePool.cpp | 12 ++-- dbms/src/Common/MemoryTracker.cpp | 4 +- dbms/src/Common/MemoryTracker.h | 5 +- dbms/src/Common/ProfileEvents.cpp | 5 ++ dbms/src/Common/Stopwatch.cpp | 14 +++++ dbms/src/Common/Stopwatch.h | 56 +++++++++++++++++++ dbms/src/Common/ThreadStatus.cpp | 11 ++-- dbms/src/Common/ThreadStatus.h | 1 + dbms/src/Common/Throttler.h | 15 ++++- .../IProfilingBlockInputStream.cpp | 14 +++++ dbms/src/IO/ReadBufferFromFileDescriptor.cpp | 11 +++- dbms/src/IO/ReadBufferFromPocoSocket.cpp | 10 ++++ dbms/src/IO/WriteBufferFromFileDescriptor.cpp | 5 ++ dbms/src/IO/WriteBufferFromPocoSocket.cpp | 12 ++++ .../MergeTree/BackgroundProcessingPool.cpp | 3 + 15 files changed, 161 insertions(+), 17 deletions(-) create mode 100644 dbms/src/Common/Stopwatch.cpp diff --git a/dbms/src/Common/BackgroundSchedulePool.cpp b/dbms/src/Common/BackgroundSchedulePool.cpp index 70a2ef66572..c27d357b91e 100644 --- a/dbms/src/Common/BackgroundSchedulePool.cpp +++ b/dbms/src/Common/BackgroundSchedulePool.cpp @@ -6,6 +6,8 @@ #include #include #include +#include "ThreadStatus.h" + namespace CurrentMetrics { @@ -230,9 +232,11 @@ void BackgroundSchedulePool::threadFunction() { setThreadName("BackgrSchedPool"); - MemoryTracker memory_tracker; - memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool); - current_memory_tracker = &memory_tracker; + if (current_thread) + { + ThreadStatus::setCurrentThreadParentQuery(nullptr); + current_thread->memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool); + } while (!shutdown) { @@ -242,8 +246,6 @@ void BackgroundSchedulePool::threadFunction() task_notification.execute(); } } - - current_memory_tracker = nullptr; } diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index 5731663b5c7..f8646702908 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -175,7 +175,7 @@ namespace CurrentMemoryTracker } } -DB::ActionBlockerSingleThread::LockHolder getCurrentMemoryTrackerBlocker() +DB::ActionLock getCurrentMemoryTrackerBlocker() { - return (DB::current_thread) ? DB::current_thread->memory_tracker.blocker.cancel() : DB::ActionBlockerSingleThread::LockHolder(nullptr); + return (DB::current_thread) ? DB::current_thread->memory_tracker.blocker.cancel() : DB::ActionLock(); } diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index 3e32ce5c9e5..5628640f9a9 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -110,7 +110,8 @@ public: void logPeakMemoryUsage() const; /// To be able to temporarily stop memory tracker - DB::ActionBlockerSingleThread blocker; + /// TODO: Use more lightweight implementation + DB::ActionBlocker blocker; }; @@ -123,4 +124,4 @@ namespace CurrentMemoryTracker } -DB::ActionBlockerSingleThread::LockHolder getCurrentMemoryTrackerBlocker(); +DB::ActionLock getCurrentMemoryTrackerBlocker(); diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index cdc09ea76b5..e72e81c9ac9 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -39,6 +39,11 @@ M(CreatedReadBufferAIO) \ M(CreatedWriteBufferOrdinary) \ M(CreatedWriteBufferAIO) \ + M(DiskReadElapsedMicroseconds) \ + M(DiskWriteElapsedMicroseconds) \ + M(NetworkReceiveElapsedMicroseconds) \ + M(NetworkSendElapsedMicroseconds) \ + M(ThrottlerSleepMicroseconds) \ \ M(ReplicatedPartFetches) \ M(ReplicatedPartFailedFetches) \ diff --git a/dbms/src/Common/Stopwatch.cpp b/dbms/src/Common/Stopwatch.cpp new file mode 100644 index 00000000000..0fb3d57ba6d --- /dev/null +++ b/dbms/src/Common/Stopwatch.cpp @@ -0,0 +1,14 @@ +#include +#include "Stopwatch.h" + +StopWatchRusage::Timestamp StopWatchRusage::Timestamp::current() +{ + StopWatchRusage::Timestamp res; + + ::rusage rusage; + ::getrusage(RUSAGE_THREAD, &rusage); + + res.user_ns = rusage.ru_utime.tv_sec * 1000000000UL + rusage.ru_utime.tv_usec; + res.sys_ns = rusage.ru_stime.tv_sec * 1000000000UL + rusage.ru_stime.tv_usec; + return res; +} diff --git a/dbms/src/Common/Stopwatch.h b/dbms/src/Common/Stopwatch.h index 414998d0d68..41d9209c11d 100644 --- a/dbms/src/Common/Stopwatch.h +++ b/dbms/src/Common/Stopwatch.h @@ -134,3 +134,59 @@ private: /// Most significant bit is a lock. When it is set, compareAndRestartDeferred method will return false. UInt64 nanoseconds() const { return StopWatchDetail::nanoseconds(clock_type) & 0x7FFFFFFFFFFFFFFFULL; } }; + + +/// Like ordinary StopWatch, but uses getrusage() system call +struct StopWatchRusage +{ + StopWatchRusage() = default; + + void start() { start_ts = Timestamp::current(); is_running = true; } + void stop() { stop_ts = Timestamp::current(); is_running = false; } + void reset() { start_ts = Timestamp(); stop_ts = Timestamp(); is_running = false; } + void restart() { start(); } + + UInt64 elapsed(bool count_user = true, bool count_sys = true) const + { + return elapsedNanoseconds(count_user, count_sys); + } + + UInt64 elapsedNanoseconds(bool count_user = true, bool count_sys = true) const + { + return (is_running ? Timestamp::current() : stop_ts).nanoseconds(count_user, count_sys) - start_ts.nanoseconds(count_user, count_sys); + } + + UInt64 elapsedMicroseconds(bool count_user = true, bool count_sys = true) const + { + return elapsedNanoseconds(count_user, count_sys) / 1000UL; + } + + UInt64 elapsedMilliseconds(bool count_user = true, bool count_sys = true) const + { + return elapsedNanoseconds(count_user, count_sys) / 1000000UL; + } + + double elapsedSeconds(bool count_user = true, bool count_sys = true) const + { + return static_cast(elapsedNanoseconds(count_user, count_sys)) / 1000000000.0; + } + +private: + + struct Timestamp + { + UInt64 user_ns = 0; + UInt64 sys_ns = 0; + + static Timestamp current(); + + UInt64 nanoseconds(bool count_user = true, bool count_sys = true) const + { + return (count_user ? user_ns : 0) + (count_sys ? sys_ns : 0); + } + }; + + Timestamp start_ts; + Timestamp stop_ts; + bool is_running = false; +}; diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index a1159897483..d2f85b1de08 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -45,7 +45,6 @@ static UInt64 getCurrentTimeMicroseconds(clockid_t clock_type = CLOCK_MONOTONIC) return ts.tv_sec * 1000000ULL + ts.tv_nsec / 1000UL; } - struct RusageCounters { /// In microseconds @@ -79,16 +78,16 @@ struct RusageCounters void setCurrent() { - rusage rusage; - getrusage(RUSAGE_THREAD, &rusage); + ::rusage rusage; + ::getrusage(RUSAGE_THREAD, &rusage); set(rusage, getCurrentTimeMicroseconds()); } void set(const ::rusage & rusage, UInt64 real_time_) { real_time = real_time_; - user_time = rusage.ru_utime.tv_sec * 1000000UL + rusage.ru_utime.tv_usec; - sys_time = rusage.ru_utime.tv_sec * 1000000UL + rusage.ru_utime.tv_usec; + user_time = rusage.ru_utime.tv_sec * 1000000UL + rusage.ru_utime.tv_usec / 1000UL; + sys_time = rusage.ru_stime.tv_sec * 1000000UL + rusage.ru_stime.tv_usec / 1000UL; page_reclaims = static_cast(rusage.ru_minflt); voluntary_context_switches = static_cast(rusage.ru_nvcsw); @@ -293,6 +292,8 @@ struct ScopeCurrentThread } }; +ThreadStatusPtr ThreadStatus::getCurrent() const { return current_thread; } + thread_local ThreadStatusPtr current_thread = ThreadStatus::create(); /// Order of current_thread and current_thread_scope matters diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index cb45c11f280..044865ddf6b 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -41,6 +41,7 @@ public: static void setCurrentThreadParentQuery(QueryStatus * parent_process); static void setCurrentThreadFromSibling(const ThreadStatusPtr & sibling_thread); + ThreadStatusPtr getCurrent() const; ~ThreadStatus(); diff --git a/dbms/src/Common/Throttler.h b/dbms/src/Common/Throttler.h index 22aa102cd1b..ba76242fbab 100644 --- a/dbms/src/Common/Throttler.h +++ b/dbms/src/Common/Throttler.h @@ -5,9 +5,16 @@ #include #include #include +#include #include +namespace ProfileEvents +{ + extern const Event ThrottlerSleepMicroseconds; +} + + namespace DB { @@ -68,10 +75,14 @@ public: if (desired_ns > elapsed_ns) { UInt64 sleep_ns = desired_ns - elapsed_ns; - timespec sleep_ts; + ::timespec sleep_ts; sleep_ts.tv_sec = sleep_ns / 1000000000; sleep_ts.tv_nsec = sleep_ns % 1000000000; - nanosleep(&sleep_ts, nullptr); /// NOTE Returns early in case of a signal. This is considered normal. + + /// NOTE: Returns early in case of a signal. This is considered normal. + ::nanosleep(&sleep_ts, nullptr); + + ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_ns / 1000UL); } } diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 6cd57d08f3e..25ca02a0398 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -1,6 +1,13 @@ #include #include #include +#include + + +namespace ProfileEvents +{ + extern const Event ThrottlerSleepMicroseconds; +} namespace DB @@ -293,6 +300,13 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0) { + /// Do not count sleeps in throttlers + if (current_thread) + { + UInt64 throttler_sleeps_ms = current_thread->performance_counters[ProfileEvents::ThrottlerSleepMicroseconds]; + total_elapsed -= static_cast(throttler_sleeps_ms) / 1000000.0; + } + if (limits.min_execution_speed && progress.rows / total_elapsed < limits.min_execution_speed) throw Exception("Query is executing too slow: " + toString(progress.rows / total_elapsed) + " rows/sec., minimum: " + toString(limits.min_execution_speed), diff --git a/dbms/src/IO/ReadBufferFromFileDescriptor.cpp b/dbms/src/IO/ReadBufferFromFileDescriptor.cpp index d2acffaf2e7..6218697795f 100644 --- a/dbms/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/dbms/src/IO/ReadBufferFromFileDescriptor.cpp @@ -14,6 +14,7 @@ namespace ProfileEvents extern const Event ReadBufferFromFileDescriptorRead; extern const Event ReadBufferFromFileDescriptorReadFailed; extern const Event ReadBufferFromFileDescriptorReadBytes; + extern const Event DiskReadElapsedMicroseconds; extern const Event Seek; } @@ -47,6 +48,7 @@ bool ReadBufferFromFileDescriptor::nextImpl() { ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorRead); + StopWatchRusage watch_ru; std::optional watch; if (profile_callback) watch.emplace(clock_type); @@ -68,6 +70,9 @@ bool ReadBufferFromFileDescriptor::nextImpl() if (res > 0) bytes_read += res; + watch_ru.stop(); + ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch_ru.elapsedMicroseconds()); + if (profile_callback) { ProfileInfo info; @@ -114,12 +119,16 @@ off_t ReadBufferFromFileDescriptor::doSeek(off_t offset, int whence) else { ProfileEvents::increment(ProfileEvents::Seek); + StopWatchRusage watch_ru; pos = working_buffer.end(); - off_t res = lseek(fd, new_pos, SEEK_SET); + off_t res = ::lseek(fd, new_pos, SEEK_SET); if (-1 == res) throwFromErrno("Cannot seek through file " + getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); pos_in_file = new_pos; + + ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch_ru.elapsedMicroseconds()); + return res; } } diff --git a/dbms/src/IO/ReadBufferFromPocoSocket.cpp b/dbms/src/IO/ReadBufferFromPocoSocket.cpp index fa20a41b004..88686b8b33c 100644 --- a/dbms/src/IO/ReadBufferFromPocoSocket.cpp +++ b/dbms/src/IO/ReadBufferFromPocoSocket.cpp @@ -4,6 +4,13 @@ #include #include +#include + + +namespace ProfileEvents +{ + extern const Event NetworkReceiveElapsedMicroseconds; +} namespace DB @@ -20,6 +27,7 @@ namespace ErrorCodes bool ReadBufferFromPocoSocket::nextImpl() { ssize_t bytes_read = 0; + StopWatchRusage watch_ru; /// Add more details to exceptions. try @@ -42,6 +50,8 @@ bool ReadBufferFromPocoSocket::nextImpl() if (bytes_read < 0) throw NetException("Cannot read from socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_READ_FROM_SOCKET); + ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch_ru.elapsedMicroseconds()); + if (bytes_read) working_buffer.resize(bytes_read); else diff --git a/dbms/src/IO/WriteBufferFromFileDescriptor.cpp b/dbms/src/IO/WriteBufferFromFileDescriptor.cpp index d22fe7229b8..75f5aefef71 100644 --- a/dbms/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/dbms/src/IO/WriteBufferFromFileDescriptor.cpp @@ -7,6 +7,7 @@ #include #include +#include namespace ProfileEvents @@ -14,6 +15,7 @@ namespace ProfileEvents extern const Event WriteBufferFromFileDescriptorWrite; extern const Event WriteBufferFromFileDescriptorWriteFailed; extern const Event WriteBufferFromFileDescriptorWriteBytes; + extern const Event DiskWriteElapsedMicroseconds; } namespace CurrentMetrics @@ -38,6 +40,8 @@ void WriteBufferFromFileDescriptor::nextImpl() if (!offset()) return; + StopWatchRusage watch_ru; + size_t bytes_written = 0; while (bytes_written != offset()) { @@ -59,6 +63,7 @@ void WriteBufferFromFileDescriptor::nextImpl() bytes_written += res; } + ProfileEvents::increment(ProfileEvents::DiskWriteElapsedMicroseconds, watch_ru.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteBytes, bytes_written); } diff --git a/dbms/src/IO/WriteBufferFromPocoSocket.cpp b/dbms/src/IO/WriteBufferFromPocoSocket.cpp index 0255f7bf5a0..cefee566055 100644 --- a/dbms/src/IO/WriteBufferFromPocoSocket.cpp +++ b/dbms/src/IO/WriteBufferFromPocoSocket.cpp @@ -4,6 +4,13 @@ #include #include +#include + + +namespace ProfileEvents +{ + extern const Event NetworkSendElapsedMicroseconds; +} namespace DB @@ -22,6 +29,8 @@ void WriteBufferFromPocoSocket::nextImpl() if (!offset()) return; + StopWatchRusage watch_ru; + size_t bytes_written = 0; while (bytes_written < offset()) { @@ -47,7 +56,10 @@ void WriteBufferFromPocoSocket::nextImpl() if (res < 0) throw NetException("Cannot write to socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_WRITE_TO_SOCKET); + bytes_written += res; + + ProfileEvents::increment(ProfileEvents::NetworkSendElapsedMicroseconds, watch_ru.elapsedMicroseconds()); } } diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index 4f41b892dd9..c4401b38547 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -115,7 +115,10 @@ void BackgroundProcessingPool::threadFunction() { setThreadName("BackgrProcPool"); if (current_thread) + { ThreadStatus::setCurrentThreadParentQuery(nullptr); + current_thread->memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool); + } pcg64 rng(randomSeed()); std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, sleep_seconds_random_part)(rng))); From e265e5c2ab04a81e96ba64f30789c825c527987a Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Tue, 29 May 2018 21:14:31 +0300 Subject: [PATCH 068/315] Refactoring. [#CLICKHOUSE-2910] --- dbms/src/Common/BackgroundSchedulePool.cpp | 10 +- dbms/src/Common/CurrentThread.cpp | 106 +++++++ dbms/src/Common/CurrentThread.h | 50 ++++ dbms/src/Common/ProfileEvents.h | 6 + dbms/src/Common/TaskStatsInfoGetter.cpp | 6 +- dbms/src/Common/TaskStatsInfoGetter.h | 1 + dbms/src/Common/ThreadStatus.cpp | 280 +++++++++--------- dbms/src/Common/ThreadStatus.h | 43 +-- .../AsynchronousBlockInputStream.h | 9 +- ...regatedMemoryEfficientBlockInputStream.cpp | 12 +- .../src/DataStreams/ParallelInputsProcessor.h | 10 +- dbms/src/Interpreters/Aggregator.cpp | 24 +- dbms/src/Interpreters/ProcessList.cpp | 16 +- dbms/src/Interpreters/ProcessList.h | 1 + .../DistributedBlockOutputStream.cpp | 13 +- .../MergeTree/BackgroundProcessingPool.cpp | 10 +- 16 files changed, 385 insertions(+), 212 deletions(-) create mode 100644 dbms/src/Common/CurrentThread.cpp create mode 100644 dbms/src/Common/CurrentThread.h diff --git a/dbms/src/Common/BackgroundSchedulePool.cpp b/dbms/src/Common/BackgroundSchedulePool.cpp index c27d357b91e..c66f0dbb087 100644 --- a/dbms/src/Common/BackgroundSchedulePool.cpp +++ b/dbms/src/Common/BackgroundSchedulePool.cpp @@ -4,9 +4,10 @@ #include #include #include +#include +#include #include #include -#include "ThreadStatus.h" namespace CurrentMetrics @@ -232,11 +233,8 @@ void BackgroundSchedulePool::threadFunction() { setThreadName("BackgrSchedPool"); - if (current_thread) - { - ThreadStatus::setCurrentThreadParentQuery(nullptr); - current_thread->memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool); - } + CurrentThread::attachQuery(nullptr); + CurrentThread::getMemoryTracker().setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool); while (!shutdown) { diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp new file mode 100644 index 00000000000..8fe35c989af --- /dev/null +++ b/dbms/src/Common/CurrentThread.cpp @@ -0,0 +1,106 @@ +#include "CurrentThread.h" +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +static ThreadStatusPtr assertCurrentThread() +{ + ThreadStatusPtr thread = current_thread; + + if (!thread) + throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR); + + if (Poco::ThreadNumber::get() != thread->poco_thread_number) + throw Exception("Current thread has different thread number", ErrorCodes::LOGICAL_ERROR); + + return thread; +} + + +void CurrentThread::attachQuery(QueryStatus * parent_process) +{ + assertCurrentThread(); + + if (!parent_process) + current_thread->attachQuery(nullptr, nullptr, nullptr); + else + current_thread->attachQuery(parent_process, &parent_process->performance_counters, &parent_process->memory_tracker); +} + + +void CurrentThread::attachQueryFromSiblingThread(const ThreadStatusPtr & sibling_thread) +{ + attachQueryFromSiblingThreadImpl(sibling_thread, true); +} + +void CurrentThread::attachQueryFromSiblingThreadIfDetached(const ThreadStatusPtr & sibling_thread) +{ + attachQueryFromSiblingThreadImpl(sibling_thread, false); +} + +void CurrentThread::updatePerformanceCounters() +{ + assertCurrentThread(); + current_thread->updatePerfomanceCountersImpl(); +} + +ThreadStatusPtr CurrentThread::get() +{ + assertCurrentThread(); + return current_thread; +} + +void CurrentThread::detachQuery() +{ + assertCurrentThread(); + current_thread->detachQuery(); +} + +bool CurrentThread::isAttachedToQuery() +{ + std::lock_guard lock(current_thread->mutex); + return current_thread->is_active_query; +} + +ProfileEvents::Counters & CurrentThread::getProfileEvents() +{ + return current_thread->performance_counters; +} + +MemoryTracker & CurrentThread::getMemoryTracker() +{ + return current_thread->memory_tracker; +} + +void CurrentThread::attachQueryFromSiblingThreadImpl(const ThreadStatusPtr & sibling_thread, bool check_detached) +{ + if (sibling_thread == nullptr) + throw Exception("Sibling thread was not initialized", ErrorCodes::LOGICAL_ERROR); + + assertCurrentThread(); + + QueryStatus * parent_query; + ProfileEvents::Counters * parent_counters; + MemoryTracker * parent_memory_tracker; + { + std::lock_guard lock(sibling_thread->mutex); + parent_query = sibling_thread->parent_query; + parent_counters = sibling_thread->performance_counters.parent; + parent_memory_tracker = sibling_thread->memory_tracker.getParent(); + } + + current_thread->attachQuery(parent_query, parent_counters, parent_memory_tracker, check_detached); +} + +} diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h new file mode 100644 index 00000000000..096e775991d --- /dev/null +++ b/dbms/src/Common/CurrentThread.h @@ -0,0 +1,50 @@ +#pragma once +#include + + +namespace ProfileEvents +{ + struct Counters; +} + +class MemoryTracker; + + +namespace DB +{ + +class QueryStatus; +class ThreadStatus; +using ThreadStatusPtr = std::shared_ptr; + + +class CurrentThread +{ +public: + + static ThreadStatusPtr get(); + + /// You must call one of these methods when create a child thread: + + /// Bundles the current thread with a query + static void attachQuery(QueryStatus * parent_process); + /// Bundles the current thread with a query bundled to the sibling thread + static void attachQueryFromSiblingThread(const ThreadStatusPtr & sibling_thread); + /// Is useful for a ThreadPool tasks + static void attachQueryFromSiblingThreadIfDetached(const ThreadStatusPtr & sibling_thread); + + static void updatePerformanceCounters(); + static bool isAttachedToQuery(); + static ProfileEvents::Counters & getProfileEvents(); + static MemoryTracker & getMemoryTracker(); + + /// Non-master threads call these method in destructor automatically + static void detachQuery(); + +private: + static void attachQueryFromSiblingThreadImpl(const ThreadStatusPtr & sibling_thread, bool check_detached = true); +}; + +} + + diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index a87df2e57cf..cbf0b11c0dc 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -66,6 +66,12 @@ namespace ProfileEvents /// Reset metrics and parent void reset(); + /// Set parent (thread unsafe) + void setParent(Counters * parent_) + { + parent = parent_; + } + /// Reset metrics void resetCounters(); diff --git a/dbms/src/Common/TaskStatsInfoGetter.cpp b/dbms/src/Common/TaskStatsInfoGetter.cpp index 0973397fa83..71bdfab32e6 100644 --- a/dbms/src/Common/TaskStatsInfoGetter.cpp +++ b/dbms/src/Common/TaskStatsInfoGetter.cpp @@ -209,12 +209,14 @@ TaskStatsInfoGetter::TaskStatsInfoGetter() throwFromErrno("Can't bind PF_NETLINK socket"); netlink_family_id = get_family_id(netlink_socket_fd); + + initial_tid = getCurrentTID(); } void TaskStatsInfoGetter::getStat(::taskstats & stat, int tid) const { if (tid < 0) - tid = getCurrentTID(); + tid = initial_tid; get_taskstats(netlink_socket_fd, netlink_family_id, tid, stat); } @@ -222,7 +224,7 @@ void TaskStatsInfoGetter::getStat(::taskstats & stat, int tid) const bool TaskStatsInfoGetter::tryGetStat(::taskstats & stat, int tid) const { if (tid < 0) - tid = getCurrentTID(); + tid = initial_tid; Exception e; return get_taskstats(netlink_socket_fd, netlink_family_id, tid, stat, &e); diff --git a/dbms/src/Common/TaskStatsInfoGetter.h b/dbms/src/Common/TaskStatsInfoGetter.h index 1d77b2fbb20..3add5055ece 100644 --- a/dbms/src/Common/TaskStatsInfoGetter.h +++ b/dbms/src/Common/TaskStatsInfoGetter.h @@ -27,6 +27,7 @@ private: int netlink_socket_fd = -1; int netlink_family_id = 0; + int initial_tid = -1; }; } diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index d2f85b1de08..8e2edc3138a 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -1,9 +1,9 @@ #include "ThreadStatus.h" #include -#include #include #include #include +#include #include @@ -31,6 +31,7 @@ namespace ProfileEvents namespace DB { + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -38,6 +39,52 @@ namespace ErrorCodes } +class CurrentThreadScope +{ +public: + + CurrentThreadScope() + { + try + { + { + std::lock_guard lock(current_thread->mutex); + current_thread->is_active_thread = true; + } + LOG_DEBUG(current_thread->log, "Thread " << current_thread->poco_thread_number << " is started"); + } + catch (...) + { + std::terminate(); + } + } + + ~CurrentThreadScope() + { + try + { + CurrentThread::detachQuery(); + LOG_DEBUG(current_thread->log, "Thread " << current_thread->poco_thread_number << " is exited"); + { + std::lock_guard lock(current_thread->mutex); + current_thread->is_active_thread = false; + } + } + catch (...) + { + std::terminate(); + } + } +}; + + +thread_local ThreadStatusPtr current_thread = ThreadStatus::create(); + +/// Order of current_thread and current_thread_scope matters +static thread_local CurrentThreadScope current_thread_scope; + + + static UInt64 getCurrentTimeMicroseconds(clockid_t clock_type = CLOCK_MONOTONIC) { struct timespec ts; @@ -62,7 +109,7 @@ struct RusageCounters set(rusage, real_time_); } - static RusageCounters zeros(UInt64 real_time) + static RusageCounters zeros(UInt64 real_time = getCurrentTimeMicroseconds()) { RusageCounters res; res.real_time = real_time; @@ -72,15 +119,10 @@ struct RusageCounters static RusageCounters current() { RusageCounters res; - res.setCurrent(); - return res; - } - - void setCurrent() - { ::rusage rusage; ::getrusage(RUSAGE_THREAD, &rusage); - set(rusage, getCurrentTimeMicroseconds()); + res.set(rusage, getCurrentTimeMicroseconds()); + return res; } void set(const ::rusage & rusage, UInt64 real_time_) @@ -106,42 +148,58 @@ struct RusageCounters static void updateProfileEvents(RusageCounters & last_counters) { - RusageCounters current = RusageCounters::current(); - RusageCounters::incrementProfileEvents(current, last_counters); - last_counters = current; + auto current_counters = current(); + incrementProfileEvents(current_counters, last_counters); + last_counters = current_counters; } }; +struct TasksStatsCounters +{ + ::taskstats stat; + + TasksStatsCounters() = default; + + static TasksStatsCounters zeros() + { + TasksStatsCounters res; + memset(&res.stat, 0, sizeof(stat)); + return res; + } + + static TasksStatsCounters current(); + + static void incrementProfileEvents(const TasksStatsCounters & curr, const TasksStatsCounters & prev) + { + ProfileEvents::increment(ProfileEvents::OSReadBytes, curr.stat.read_bytes - prev.stat.read_bytes); + ProfileEvents::increment(ProfileEvents::OSWriteBytes, curr.stat.write_bytes - prev.stat.write_bytes); + ProfileEvents::increment(ProfileEvents::OSReadChars, curr.stat.read_char - prev.stat.read_char); + ProfileEvents::increment(ProfileEvents::OSWriteChars, curr.stat.write_char - prev.stat.write_char); + } + + static void updateProfileEvents(TasksStatsCounters & last_counters) + { + auto current_counters = current(); + incrementProfileEvents(current_counters, last_counters); + last_counters = current_counters; + } +}; struct ThreadStatus::Impl { RusageCounters last_rusage; + TasksStatsCounters last_taskstats; TaskStatsInfoGetter info_getter; }; -//static void QueryThreadStatusOnThreadFinish(void * arg) -//{ -// auto thread_status = static_cast(arg); -// thread_status->is_finished = true; -// LOG_DEBUG(thread_status->log, "Thread " << thread_status->poco_thread_number << " is finished"); -//} - -static pthread_once_t once_query_at_exit_callback = PTHREAD_ONCE_INIT; -static pthread_key_t tid_key_at_exit; - -static void thread_destructor(void * data) +TasksStatsCounters TasksStatsCounters::current() { - auto thread_status = static_cast(data); - thread_status->onExit(); - LOG_DEBUG(thread_status->log, "Destruct thread " << thread_status->poco_thread_number); - thread_status->thread_exited = true; + TasksStatsCounters res; + current_thread->impl->info_getter.getStat(res.stat, current_thread->os_thread_id); + return res; } -static void thread_create_at_exit_key() { - if (0 != pthread_key_create(&tid_key_at_exit, thread_destructor)) - throw Exception("Failed pthread_key_create", ErrorCodes::PTHREAD_ERROR); -} ThreadStatus::ThreadStatus() @@ -150,45 +208,44 @@ ThreadStatus::ThreadStatus() os_thread_id(TaskStatsInfoGetter::getCurrentTID()), log(&Poco::Logger::get("ThreadStatus")) { - impl = std::make_shared(); + impl = std::make_unique(); LOG_DEBUG(log, "Thread " << poco_thread_number << " created"); +} - if (0 != pthread_once(&once_query_at_exit_callback, thread_create_at_exit_key)) - throw Exception("Failed pthread_once", ErrorCodes::PTHREAD_ERROR); - - if (nullptr != pthread_getspecific(tid_key_at_exit)) - throw Exception("pthread_getspecific is already set", ErrorCodes::LOGICAL_ERROR); - - if (0 != pthread_setspecific(tid_key_at_exit, static_cast(this))) - throw Exception("Failed pthread_setspecific", ErrorCodes::PTHREAD_ERROR); +ThreadStatusPtr ThreadStatus::create() +{ + return ThreadStatusPtr(new ThreadStatus); } ThreadStatus::~ThreadStatus() { - LOG_DEBUG(log, "Thread " << poco_thread_number << " destroyed"); + LOG_DEBUG(log, "Thread " << poco_thread_number << " destroyed in " << Poco::ThreadNumber::get()); } -void ThreadStatus::init(QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker) +void ThreadStatus::attachQuery( + QueryStatus *parent_query_, + ProfileEvents::Counters *parent_counters, + MemoryTracker *parent_memory_tracker, + bool check_detached) { - if (initialized) - { - if (auto counters_parent = performance_counters.parent) - if (counters_parent != parent_counters) - LOG_WARNING(log, "Parent performance counters are already set, overwrite"); + std::lock_guard lock(mutex); - if (auto tracker_parent = memory_tracker.getParent()) - if (tracker_parent != parent_memory_tracker) - LOG_WARNING(log, "Parent memory tracker is already set, overwrite"); + if (check_detached && is_active_query) + throw Exception("Query is already active", ErrorCodes::LOGICAL_ERROR); - return; - } + if (auto counters_parent = performance_counters.parent) + if (counters_parent != parent_counters) + LOG_WARNING(log, "Parent performance counters are already set, overwrite"); + + if (auto tracker_parent = memory_tracker.getParent()) + if (tracker_parent != parent_memory_tracker) + LOG_WARNING(log, "Parent memory tracker is already set, overwrite"); parent_query = parent_query_; performance_counters.parent = parent_counters; memory_tracker.setParent(parent_memory_tracker); memory_tracker.setDescription("(for thread)"); - initialized = true; /// Attach current thread to list of query threads if (parent_query) @@ -200,30 +257,28 @@ void ThreadStatus::init(QueryStatus * parent_query_, ProfileEvents::Counters * p throw Exception("Thread " + std::to_string(current_thread->poco_thread_number) + " is set twice", ErrorCodes::LOGICAL_ERROR); } - onStart(); -} - -void ThreadStatus::onStart() -{ /// First init of thread rusage counters, set real time to zero, other metrics remain as is - impl->last_rusage.setCurrent(); - RusageCounters::incrementProfileEvents(impl->last_rusage, RusageCounters::zeros(impl->last_rusage.real_time)); + if (is_first_query_of_the_thread) + { + impl->last_rusage = RusageCounters::zeros(); + impl->last_taskstats = TasksStatsCounters::zeros(); + updatePerfomanceCountersImpl(); + } + else + { + impl->last_rusage = RusageCounters::current(); + impl->last_taskstats = TasksStatsCounters::current(); + } + + is_active_query = true; } -void ThreadStatus::onExit() +void ThreadStatus::updatePerfomanceCountersImpl() { - RusageCounters::updateProfileEvents(impl->last_rusage); - try { - ::taskstats stat; - TaskStatsInfoGetter info_getter; - info_getter.getStat(stat); - - ProfileEvents::increment(ProfileEvents::OSReadBytes, stat.read_bytes); - ProfileEvents::increment(ProfileEvents::OSWriteBytes, stat.write_bytes); - ProfileEvents::increment(ProfileEvents::OSReadChars, stat.read_char); - ProfileEvents::increment(ProfileEvents::OSWriteChars, stat.write_char); + RusageCounters::incrementProfileEvents(RusageCounters::current(), impl->last_rusage); + TasksStatsCounters::incrementProfileEvents(TasksStatsCounters::current(), impl->last_taskstats); } catch (...) { @@ -231,75 +286,34 @@ void ThreadStatus::onExit() } } +void ThreadStatus::detachQuery() +{ + std::lock_guard lock(mutex); + if (!is_active_query) + return; + + updatePerfomanceCountersImpl(); + + is_first_query_of_the_thread = false; + is_active_query = false; + + /// Detach from parent + performance_counters.setParent(nullptr); + memory_tracker.setParent(nullptr); +} + void ThreadStatus::reset() { std::lock_guard lock(mutex); - initialized = false; - parent_query = nullptr; + if (is_active_query) + throw Exception("Query is still active", ErrorCodes::LOGICAL_ERROR); + performance_counters.reset(); memory_tracker.reset(); + + performance_counters.setParent(nullptr); memory_tracker.setParent(nullptr); } - -void ThreadStatus::setCurrentThreadParentQuery(QueryStatus * parent_process) -{ - if (!current_thread) - throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR); - - if (!parent_process) - { - current_thread->init(nullptr, nullptr, nullptr); - return; - } - - current_thread->init(parent_process, &parent_process->performance_counters, &parent_process->memory_tracker); -} - -void ThreadStatus::setCurrentThreadFromSibling(const ThreadStatusPtr & sibling_thread) -{ - if (!current_thread) - throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR); - - if (sibling_thread == nullptr) - throw Exception("Sibling thread was not initialized", ErrorCodes::LOGICAL_ERROR); - - std::lock_guard lock(sibling_thread->mutex); - current_thread->init(sibling_thread->parent_query, sibling_thread->performance_counters.parent, sibling_thread->memory_tracker.getParent()); -} - - -struct ScopeCurrentThread -{ - ScopeCurrentThread() - { - if (!current_thread) - std::terminate(); // current_thread must be initialized - } - - ~ScopeCurrentThread() - { - if (!current_thread) - std::terminate(); // current_thread must be initialized - - if (Poco::ThreadNumber::get() != current_thread->poco_thread_number) - std::terminate(); // unexpected thread number - - current_thread->onExit(); - LOG_DEBUG(current_thread->log, "Thread " << current_thread->poco_thread_number << " is exiting"); - current_thread->thread_exited = true; - } -}; - -ThreadStatusPtr ThreadStatus::getCurrent() const { return current_thread; } - -thread_local ThreadStatusPtr current_thread = ThreadStatus::create(); - -/// Order of current_thread and current_thread_scope matters -static thread_local ScopeCurrentThread current_thread_scope; - - - - } diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 044865ddf6b..7992627087e 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -2,7 +2,6 @@ #include #include #include -#include #include @@ -17,11 +16,14 @@ namespace DB class QueryStatus; class ThreadStatus; -struct ScopeCurrentThread; +class ScopeCurrentThread; using ThreadStatusPtr = std::shared_ptr; -class ThreadStatus : public ext::shared_ptr_helper +extern thread_local ThreadStatusPtr current_thread; + + +class ThreadStatus { public: @@ -32,32 +34,39 @@ public: int os_thread_id = -1; std::mutex mutex; - void init(QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker); - void onStart(); - void onExit(); +public: + + /// A constructor + static ThreadStatusPtr create(); /// Reset all references and metrics void reset(); - static void setCurrentThreadParentQuery(QueryStatus * parent_process); - static void setCurrentThreadFromSibling(const ThreadStatusPtr & sibling_thread); - ThreadStatusPtr getCurrent() const; - ~ThreadStatus(); - friend struct ScopeCurrentThread; +protected: -//protected: ThreadStatus(); + void attachQuery( + QueryStatus *parent_query_, + ProfileEvents::Counters *parent_counters, + MemoryTracker *parent_memory_tracker, + bool check_detached = true); + void detachQuery(); + void updatePerfomanceCountersImpl(); - bool initialized = false; - bool thread_exited = false; + bool is_active_query = false; + bool is_active_thread = false; + bool is_first_query_of_the_thread = true; Poco::Logger * log; + friend class CurrentThreadScope; + friend class CurrentThread; + friend struct TasksStatsCounters; + struct Impl; - std::shared_ptr impl; + std::unique_ptr impl; + Impl & getImpl() { return *impl; } }; -extern thread_local ThreadStatusPtr current_thread; - } diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index d74bea070f1..0d7b1075c9d 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include @@ -99,7 +99,7 @@ protected: /// If there were no calculations yet, calculate the first block synchronously if (!started) { - ThreadStatusPtr main_thread = current_thread; + auto main_thread = CurrentThread::get(); calculate(main_thread); started = true; } @@ -124,7 +124,7 @@ protected: void next() { ready.reset(); - pool.schedule([this, main_thread=current_thread] () { calculate(main_thread); }); + pool.schedule([this, main_thread=CurrentThread::get()] () { calculate(main_thread); }); } @@ -132,14 +132,13 @@ protected: void calculate(ThreadStatusPtr main_thread) { CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); try { if (first) { first = false; - if (main_thread) - ThreadStatus::setCurrentThreadFromSibling(main_thread); setThreadName("AsyncBlockInput"); children.back()->readPrefix(); } diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index b65b5eb7e36..b999e5760c8 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace CurrentMetrics @@ -176,11 +176,10 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start() { auto & child = children[i]; - auto main_thread = current_thread; + auto main_thread = CurrentThread::get(); reading_pool->schedule([&child, main_thread] { - if (main_thread) - ThreadStatus::setCurrentThreadFromSibling(main_thread); + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); setThreadName("MergeAggReadThr"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; child->readPrefix(); @@ -297,7 +296,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::finalize() void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadStatusPtr main_thread) { if (main_thread) - ThreadStatus::setCurrentThreadFromSibling(main_thread); + CurrentThread::attachQueryFromSiblingThread(main_thread); setThreadName("MergeAggMergThr"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; @@ -485,8 +484,7 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate auto main_thread = current_thread; reading_pool->schedule([&input, &read_from_input, main_thread] { - if (main_thread) - ThreadStatus::setCurrentThreadFromSibling(main_thread); + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); setThreadName("MergeAggReadThr"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; read_from_input(input); diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index fdddf6b2149..1af47062b13 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include /** Allows to process multiple block input streams (sources) in parallel, using specified number of threads. @@ -104,11 +104,11 @@ public: /// Start background threads, start work. void process() { - QueryStatus * current_query = current_thread ? current_thread->parent_query : nullptr; active_threads = max_threads; threads.reserve(max_threads); + auto main_thread = CurrentThread::get(); for (size_t i = 0; i < max_threads; ++i) - threads.emplace_back([=] () { thread(current_query, i); } ); + threads.emplace_back([=] () { thread(main_thread, i); } ); } /// Ask all sources to stop earlier than they run out. @@ -176,9 +176,9 @@ private: } } - void thread(QueryStatus * query_status, size_t thread_num) + void thread(ThreadStatusPtr main_thread, size_t thread_num) { - ThreadStatus::setCurrentThreadParentQuery(query_status); + CurrentThread::attachQueryFromSiblingThread(main_thread); std::exception_ptr exception; setThreadName("ParalInputsProc"); diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index e2b36ed6d60..65bb2b3a380 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -21,10 +21,10 @@ #include #include #include +#include #include #include #include -#include "Common/ThreadStatus.h" namespace ProfileEvents @@ -141,9 +141,8 @@ Aggregator::Aggregator(const Params & params_) isCancelled([]() { return false; }) { /// Use query-level memory tracker - if (current_thread) - if (auto memory_tracker = current_thread->memory_tracker.getParent()) - memory_usage_before_aggregation = memory_tracker->get(); + if (auto memory_tracker = CurrentThread::getMemoryTracker().getParent()) + memory_usage_before_aggregation = memory_tracker->get(); aggregate_functions.resize(params.aggregates_size); for (size_t i = 0; i < params.aggregates_size; ++i) @@ -801,9 +800,8 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re size_t result_size = result.sizeWithoutOverflowRow(); Int64 current_memory_usage = 0; - if (current_thread) - if (auto memory_tracker = current_thread->memory_tracker.getParent()) - current_memory_usage = memory_tracker->get(); + if (auto memory_tracker = CurrentThread::getMemoryTracker().getParent()) + current_memory_usage = memory_tracker->get(); auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation; /// Here all the results in the sum are taken into account, from different threads. @@ -1278,7 +1276,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( { auto converter = [&](size_t bucket, ThreadStatusPtr main_thread) { - ThreadStatus::setCurrentThreadFromSibling(main_thread); + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); return convertOneBucketToBlock(data_variants, method, final, bucket); }; @@ -1293,7 +1291,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( if (method.data.impls[bucket].empty()) continue; - tasks[bucket] = std::packaged_task(std::bind(converter, bucket, current_thread)); + tasks[bucket] = std::packaged_task(std::bind(converter, bucket, CurrentThread::get())); if (thread_pool) thread_pool->schedule([bucket, &tasks] { tasks[bucket](); }); @@ -1723,13 +1721,13 @@ private: return; parallel_merge_data->pool.schedule(std::bind(&MergingAndConvertingBlockInputStream::thread, this, - max_scheduled_bucket_num, current_thread)); + max_scheduled_bucket_num, CurrentThread::get())); } void thread(Int32 bucket_num, ThreadStatusPtr main_thread) { - ThreadStatus::setCurrentThreadFromSibling(main_thread); setThreadName("MergingAggregtd"); + CurrentThread::attachQueryFromSiblingThread(main_thread); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; try @@ -2035,7 +2033,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadStatusPtr main_thread) { - ThreadStatus::setCurrentThreadFromSibling(main_thread); + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); for (Block & block : bucket_to_blocks[bucket]) { @@ -2069,7 +2067,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV result.aggregates_pools.push_back(std::make_shared()); Arena * aggregates_pool = result.aggregates_pools.back().get(); - auto task = std::bind(merge_bucket, bucket, aggregates_pool, current_thread); + auto task = std::bind(merge_bucket, bucket, aggregates_pool, CurrentThread::get()); if (thread_pool) thread_pool->schedule(task); diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 5f5f0242bcd..125df365dda 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -165,15 +166,10 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as /// Query-level memory tracker is already set in the QueryStatus constructor - if (!current_thread) - throw Exception("Thread is not initialized", ErrorCodes::LOGICAL_ERROR); - - if (current_thread) - { - current_thread->setCurrentThreadParentQuery(&*process_it); - current_thread->memory_tracker.setOrRaiseLimit(settings.max_memory_usage); - current_thread->memory_tracker.setDescription("(for thread)"); - } + /// Attach master thread + CurrentThread::attachQuery(&*process_it); + current_thread->memory_tracker.setOrRaiseLimit(settings.max_memory_usage); + current_thread->memory_tracker.setDescription("(for thread)"); if (!user_process_list.user_throttler) { @@ -201,7 +197,7 @@ ProcessListEntry::~ProcessListEntry() /// Finalize all threads statuses { - current_thread->onExit(); + CurrentThread::detachQuery(); std::lock_guard lock(it->threads_mutex); diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index ac0958a9079..2519d1264e7 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -78,6 +78,7 @@ class QueryStatus protected: friend class ProcessList; friend class ThreadStatus; + friend class CurrentThread; friend class ProcessListEntry; String query; diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 461edfb98e2..ca112a5abce 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -32,7 +33,6 @@ #include #include #include -#include @@ -192,9 +192,12 @@ void DistributedBlockOutputStream::waitForJobs() ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutputStream::JobReplica & job, const Block & current_block) { - auto main_thread = current_thread; + auto main_thread = CurrentThread::get(); return [this, main_thread, &job, ¤t_block]() { + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + setThreadName("DistrOutStrProc"); + ++job.blocks_started; SCOPE_EXIT({ @@ -205,12 +208,6 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp job.max_elapsed_time_for_block_ms = std::max(job.max_elapsed_time_for_block_ms, elapsed_time_for_block_ms); }); - if (current_thread) - { - ThreadStatus::setCurrentThreadFromSibling(main_thread); - setThreadName("DistrOutStrProc"); - } - const auto & shard_info = cluster->getShardsInfo()[job.shard_index]; size_t num_shards = cluster->getShardsInfo().size(); auto & shard_job = per_shard_jobs[job.shard_index]; diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index c4401b38547..45bcb2c3810 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include @@ -114,11 +114,9 @@ BackgroundProcessingPool::~BackgroundProcessingPool() void BackgroundProcessingPool::threadFunction() { setThreadName("BackgrProcPool"); - if (current_thread) - { - ThreadStatus::setCurrentThreadParentQuery(nullptr); - current_thread->memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool); - } + + CurrentThread::attachQuery(nullptr); + CurrentThread::getMemoryTracker().setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool); pcg64 rng(randomSeed()); std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, sleep_seconds_random_part)(rng))); From 0355f81f206a9771b0a5abf82bab7695aa0a8edb Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Thu, 31 May 2018 18:54:08 +0300 Subject: [PATCH 069/315] [WIP] Introduced query_thread_log. [#CLICKHOUSE-2910] --- dbms/src/Common/ActionBlocker.h | 14 +- dbms/src/Common/ActionLock.h | 2 +- dbms/src/Common/CurrentThread.cpp | 28 +- dbms/src/Common/CurrentThread.h | 9 +- dbms/src/Common/MemoryTracker.cpp | 16 +- dbms/src/Common/MemoryTracker.h | 7 +- dbms/src/Common/ProfileEvents.cpp | 28 +- dbms/src/Common/ProfileEvents.h | 15 +- dbms/src/Common/SimpleActionBlocker.h | 79 ++++++ dbms/src/Common/Stopwatch.h | 8 +- dbms/src/Common/ThreadStatus.cpp | 257 ++++++++++++------ dbms/src/Common/ThreadStatus.h | 67 ++++- .../AsynchronousBlockInputStream.h | 11 +- .../IProfilingBlockInputStream.cpp | 9 +- dbms/src/IO/ReadBufferFromFileDescriptor.cpp | 13 +- dbms/src/IO/ReadBufferFromPocoSocket.cpp | 5 +- dbms/src/IO/WriteBufferFromFileDescriptor.cpp | 4 +- dbms/src/IO/WriteBufferFromPocoSocket.cpp | 6 +- dbms/src/Interpreters/Context.cpp | 44 +-- dbms/src/Interpreters/Context.h | 2 + dbms/src/Interpreters/ExternalLoader.cpp | 2 +- dbms/src/Interpreters/ProcessList.cpp | 15 +- dbms/src/Interpreters/ProcessList.h | 16 +- dbms/src/Interpreters/QueryLog.cpp | 139 +++++----- dbms/src/Interpreters/QueryLog.h | 2 + dbms/src/Interpreters/QueryThreadLog.cpp | 108 ++++++++ dbms/src/Interpreters/QueryThreadLog.h | 53 ++++ dbms/src/Interpreters/Settings.h | 5 +- dbms/src/Interpreters/SystemLog.cpp | 1 + dbms/src/Interpreters/SystemLog.h | 27 +- .../Interpreters/ThreadPerformanceProfile.h | 23 -- .../Storages/MergeTree/MergeTreeReader.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 2 +- dbms/src/Storages/StorageBuffer.cpp | 2 +- .../System/StorageSystemProcesses.cpp | 8 +- 35 files changed, 717 insertions(+), 312 deletions(-) create mode 100644 dbms/src/Common/SimpleActionBlocker.h create mode 100644 dbms/src/Interpreters/QueryThreadLog.cpp create mode 100644 dbms/src/Interpreters/QueryThreadLog.h delete mode 100644 dbms/src/Interpreters/ThreadPerformanceProfile.h diff --git a/dbms/src/Common/ActionBlocker.h b/dbms/src/Common/ActionBlocker.h index c1b241a27ad..8b65d181a5d 100644 --- a/dbms/src/Common/ActionBlocker.h +++ b/dbms/src/Common/ActionBlocker.h @@ -1,21 +1,22 @@ #pragma once - #include #include #include + namespace DB { -/// An atomic variable that is used to block and interrupt certain actions -/// If it is not zero then actions related with it should be considered as interrupted +/// An atomic variable that is used to block and interrupt certain actions. +/// If it is not zero then actions related with it should be considered as interrupted. +/// Uses shared_ptr and the lock uses weak_ptr to be able to "hold" a lock when an object with blocker has already died. class ActionBlocker { private: using Counter = std::atomic; using CounterPtr = std::shared_ptr; - mutable CounterPtr counter; + CounterPtr counter; public: ActionBlocker() : counter(std::make_shared(0)) {} @@ -30,7 +31,10 @@ public: void cancelForever() const { ++(*counter); } /// Returns reference to counter to allow to watch on it directly. - auto & getCounter() { return *counter; } + Counter & getCounter() { return *counter; } }; + + + } diff --git a/dbms/src/Common/ActionLock.h b/dbms/src/Common/ActionLock.h index 3d6bfc8ada7..1167a23b8dd 100644 --- a/dbms/src/Common/ActionLock.h +++ b/dbms/src/Common/ActionLock.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include + namespace DB { diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 8fe35c989af..6f11ab93309 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -1,8 +1,9 @@ #include "CurrentThread.h" -#include -#include #include +#include #include +#include +#include namespace DB @@ -69,7 +70,6 @@ void CurrentThread::detachQuery() bool CurrentThread::isAttachedToQuery() { - std::lock_guard lock(current_thread->mutex); return current_thread->is_active_query; } @@ -83,8 +83,20 @@ MemoryTracker & CurrentThread::getMemoryTracker() return current_thread->memory_tracker; } -void CurrentThread::attachQueryFromSiblingThreadImpl(const ThreadStatusPtr & sibling_thread, bool check_detached) +void CurrentThread::updateProgressIn(const Progress & value) { + current_thread->progress_in.incrementPiecewiseAtomically(value); +} + +void CurrentThread::updateProgressOut(const Progress & value) +{ + current_thread->progress_out.incrementPiecewiseAtomically(value); +} + +void CurrentThread::attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thread, bool check_detached) +{ + LOG_DEBUG(&Poco::Logger::get("CurrentThread"), __PRETTY_FUNCTION__ << ":" << __LINE__ << " " << "?" << " " << sibling_thread.get()); + if (sibling_thread == nullptr) throw Exception("Sibling thread was not initialized", ErrorCodes::LOGICAL_ERROR); @@ -94,12 +106,16 @@ void CurrentThread::attachQueryFromSiblingThreadImpl(const ThreadStatusPtr & sib ProfileEvents::Counters * parent_counters; MemoryTracker * parent_memory_tracker; { + /// NOTE: It is almost the only place where ThreadStatus::mutex is required + /// In other cases ThreadStatus must be accessed from the current_thread std::lock_guard lock(sibling_thread->mutex); - parent_query = sibling_thread->parent_query; - parent_counters = sibling_thread->performance_counters.parent; + parent_query = sibling_thread->getParentQuery(); + parent_counters = sibling_thread->performance_counters.getParent(); parent_memory_tracker = sibling_thread->memory_tracker.getParent(); } + LOG_DEBUG(&Poco::Logger::get("CurrentThread"), __PRETTY_FUNCTION__ << ":" << __LINE__ << " " << parent_query << " " << sibling_thread.get()); + current_thread->attachQuery(parent_query, parent_counters, parent_memory_tracker, check_detached); } diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index 096e775991d..95046a458ca 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -4,7 +4,7 @@ namespace ProfileEvents { - struct Counters; +class Counters; } class MemoryTracker; @@ -15,6 +15,7 @@ namespace DB class QueryStatus; class ThreadStatus; +struct Progress; using ThreadStatusPtr = std::shared_ptr; @@ -37,12 +38,14 @@ public: static bool isAttachedToQuery(); static ProfileEvents::Counters & getProfileEvents(); static MemoryTracker & getMemoryTracker(); + static void updateProgressIn(const Progress & value); + static void updateProgressOut(const Progress & value); - /// Non-master threads call these method in destructor automatically + /// Non-master threads call this method in destructor automatically static void detachQuery(); private: - static void attachQueryFromSiblingThreadImpl(const ThreadStatusPtr & sibling_thread, bool check_detached = true); + static void attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thread, bool check_detached = true); }; } diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index f8646702908..dc8f86c78f7 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -1,9 +1,10 @@ +#include "MemoryTracker.h" #include #include #include #include +#include #include -#include namespace DB @@ -158,24 +159,21 @@ namespace CurrentMemoryTracker { void alloc(Int64 size) { - if (DB::current_thread) - DB::current_thread->memory_tracker.alloc(size); + DB::CurrentThread::getMemoryTracker().alloc(size); } void realloc(Int64 old_size, Int64 new_size) { - if (DB::current_thread) - DB::current_thread->memory_tracker.alloc(new_size - old_size); + DB::CurrentThread::getMemoryTracker().alloc(new_size - old_size); } void free(Int64 size) { - if (DB::current_thread) - DB::current_thread->memory_tracker.free(size); + DB::CurrentThread::getMemoryTracker().free(size); } } -DB::ActionLock getCurrentMemoryTrackerBlocker() +DB::SimpleActionLock getCurrentMemoryTrackerActionLock() { - return (DB::current_thread) ? DB::current_thread->memory_tracker.blocker.cancel() : DB::ActionLock(); + return DB::CurrentThread::getMemoryTracker().blocker.cancel(); } diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index 5628640f9a9..63e21da6ac1 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace CurrentMetrics @@ -110,8 +110,7 @@ public: void logPeakMemoryUsage() const; /// To be able to temporarily stop memory tracker - /// TODO: Use more lightweight implementation - DB::ActionBlocker blocker; + DB::SimpleActionBlocker blocker; }; @@ -124,4 +123,4 @@ namespace CurrentMemoryTracker } -DB::ActionLock getCurrentMemoryTrackerBlocker(); +DB::SimpleActionLock getCurrentMemoryTrackerActionLock(); diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index e72e81c9ac9..882e6faa9ea 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include @@ -151,16 +151,18 @@ M(NetworkErrors) \ \ M(RealTimeMicroseconds) \ - M(RusageUserTimeMicroseconds) \ - M(RusageSystemTimeMicroseconds) \ - M(RusagePageReclaims) \ - M(RusagePageVoluntaryContextSwitches) \ - M(RusagePageInvoluntaryContextSwitches) \ + M(UserTimeMicroseconds) \ + M(SystemTimeMicroseconds) \ + M(SoftPageFaults) \ + M(HardPageFaults) \ + M(VoluntaryContextSwitches) \ + M(InvoluntaryContextSwitches) \ \ + M(OSIOWaitMicroseconds) \ M(OSReadBytes) \ M(OSWriteBytes) \ M(OSReadChars) \ - M(OSWriteChars) + M(OSWriteChars) \ namespace ProfileEvents @@ -180,8 +182,9 @@ const Event Counters::num_counters = END; Counters::Counters(Level level, Counters * parent) - : parent(parent), level(level), - counters_holder(new Counter[num_counters] {}) + : parent(parent), + counters_holder(new Counter[num_counters] {}), + level(level) { counters = counters_holder.get(); } @@ -204,7 +207,7 @@ void Counters::reset() void Counters::getPartiallyAtomicSnapshot(Counters & res) const { for (Event i = 0; i < num_counters; ++i) - res.counters[i].store(counters[i], std::memory_order_relaxed); + res.counters[i].store(counters[i].load(std::memory_order_relaxed), std::memory_order_relaxed); } const char * getDescription(Event event) @@ -225,10 +228,7 @@ Event end() { return END; } void increment(Event event, Count amount) { - if (DB::current_thread) - DB::current_thread->performance_counters.increment(event, amount); - else - global_counters.increment(event, amount); + DB::CurrentThread::getProfileEvents().increment(event, amount); } void Counters::dumpToArrayColumns(DB::IColumn * column_names_, DB::IColumn * column_values_, bool nonzero_only) diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index cbf0b11c0dc..31f269b422c 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -31,16 +31,17 @@ namespace ProfileEvents Thread }; - struct Counters + class Counters { Counter * counters = nullptr; Counters * parent = nullptr; - Level level = Level::Thread; std::unique_ptr counters_holder; - Counters() = default; + public: - Counters(Level level, Counters * parent = nullptr); + Level level = Level::Thread; + + Counters(Level level = Level::Thread, Counters * parent = nullptr); /// Global level static initializer Counters(Counter * allocated_counters) @@ -66,6 +67,12 @@ namespace ProfileEvents /// Reset metrics and parent void reset(); + /// Get parent (thread unsafe) + Counters * getParent() + { + return parent; + } + /// Set parent (thread unsafe) void setParent(Counters * parent_) { diff --git a/dbms/src/Common/SimpleActionBlocker.h b/dbms/src/Common/SimpleActionBlocker.h new file mode 100644 index 00000000000..4a96db0e09d --- /dev/null +++ b/dbms/src/Common/SimpleActionBlocker.h @@ -0,0 +1,79 @@ +#pragma once +#include + + +namespace DB +{ + +class SimpleActionLock; + + +/// Similar to ActionBlocker, but without weak_ptr magic +class SimpleActionBlocker +{ + using Counter = std::atomic; + Counter counter = 0; + +public: + + SimpleActionBlocker() = default; + + bool isCancelled() const { return counter > 0; } + + /// Temporarily blocks corresponding actions (while the returned object is alive) + friend class SimpleActionLock; + inline SimpleActionLock cancel(); + + /// Cancel the actions forever. + void cancelForever() { ++counter; } +}; + + +/// Blocks related action while a SimpleActionLock instance exists +class SimpleActionLock +{ + SimpleActionBlocker * block = nullptr; + +public: + + SimpleActionLock() = default; + + explicit SimpleActionLock(SimpleActionBlocker & block_) : block(&block_) + { + ++block->counter; + } + + SimpleActionLock(const SimpleActionLock &) = delete; + + SimpleActionLock(SimpleActionLock && rhs) noexcept + { + *this = std::move(rhs); + } + + SimpleActionLock & operator=(const SimpleActionLock &) = delete; + + SimpleActionLock & operator=(SimpleActionLock && rhs) noexcept + { + if (block) + --block->counter; + + block = rhs.block; + rhs.block = nullptr; + + return *this; + } + + ~SimpleActionLock() + { + if (block) + --block->counter; + } +}; + + +SimpleActionLock SimpleActionBlocker::cancel() +{ + return SimpleActionLock(*this); +} + +} diff --git a/dbms/src/Common/Stopwatch.h b/dbms/src/Common/Stopwatch.h index 41d9209c11d..223e1255392 100644 --- a/dbms/src/Common/Stopwatch.h +++ b/dbms/src/Common/Stopwatch.h @@ -35,9 +35,11 @@ public: void stop() { stop_ns = nanoseconds(); is_running = false; } void reset() { start_ns = 0; stop_ns = 0; is_running = false; } void restart() { start(); } - UInt64 elapsed() const { return is_running ? nanoseconds() - start_ns : stop_ns - start_ns; } - UInt64 elapsedMilliseconds() const { return elapsed() / 1000000UL; } - double elapsedSeconds() const { return static_cast(elapsed()) / 1000000000ULL; } + UInt64 elapsed() const { return elapsedNanoseconds(); } + UInt64 elapsedNanoseconds() const { return is_running ? nanoseconds() - start_ns : stop_ns - start_ns; } + UInt64 elapsedMicroseconds() const { return elapsedNanoseconds() / 1000U; } + UInt64 elapsedMilliseconds() const { return elapsedNanoseconds() / 1000000UL; } + double elapsedSeconds() const { return static_cast(elapsedNanoseconds()) / 1000000000ULL; } private: UInt64 start_ns = 0; diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 8e2edc3138a..275b4c7720c 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -1,9 +1,11 @@ #include "ThreadStatus.h" #include #include -#include #include #include +#include +#include +#include #include @@ -15,12 +17,14 @@ namespace ProfileEvents { extern const Event RealTimeMicroseconds; - extern const Event RusageUserTimeMicroseconds; - extern const Event RusageSystemTimeMicroseconds; - extern const Event RusagePageReclaims; - extern const Event RusagePageVoluntaryContextSwitches; - extern const Event RusagePageInvoluntaryContextSwitches; + extern const Event UserTimeMicroseconds; + extern const Event SystemTimeMicroseconds; + extern const Event SoftPageFaults; + extern const Event HardPageFaults; + extern const Event VoluntaryContextSwitches; + extern const Event InvoluntaryContextSwitches; + extern const Event OSIOWaitMicroseconds; extern const Event OSReadBytes; extern const Event OSWriteBytes; extern const Event OSReadChars; @@ -39,7 +43,8 @@ namespace ErrorCodes } -class CurrentThreadScope +/// Implicitly finalizes current thread in the destructor +class ThreadStatus::CurrentThreadScope { public: @@ -81,75 +86,83 @@ public: thread_local ThreadStatusPtr current_thread = ThreadStatus::create(); /// Order of current_thread and current_thread_scope matters -static thread_local CurrentThreadScope current_thread_scope; +static thread_local ThreadStatus::CurrentThreadScope current_thread_scope; +/// Handles overflow +template +inline TUInt safeDiff(TUInt prev, TUInt curr) +{ + return curr >= prev ? curr - prev : 0; +} -static UInt64 getCurrentTimeMicroseconds(clockid_t clock_type = CLOCK_MONOTONIC) + +static UInt64 getCurrentTimeNanoseconds(clockid_t clock_type = CLOCK_MONOTONIC) { struct timespec ts; clock_gettime(clock_type, &ts); return ts.tv_sec * 1000000ULL + ts.tv_nsec / 1000UL; } + struct RusageCounters { - /// In microseconds + /// In nanoseconds UInt64 real_time = 0; UInt64 user_time = 0; UInt64 sys_time = 0; - UInt64 page_reclaims = 0; + UInt64 soft_page_faults = 0; + UInt64 hard_page_faults = 0; UInt64 voluntary_context_switches = 0; UInt64 involuntary_context_switches = 0; RusageCounters() = default; - RusageCounters(const ::rusage & rusage, UInt64 real_time_) + RusageCounters(const ::rusage & rusage_, UInt64 real_time_) { - set(rusage, real_time_); - } - - static RusageCounters zeros(UInt64 real_time = getCurrentTimeMicroseconds()) - { - RusageCounters res; - res.real_time = real_time; - return res; - } - - static RusageCounters current() - { - RusageCounters res; - ::rusage rusage; - ::getrusage(RUSAGE_THREAD, &rusage); - res.set(rusage, getCurrentTimeMicroseconds()); - return res; + set(rusage_, real_time_); } void set(const ::rusage & rusage, UInt64 real_time_) { real_time = real_time_; - user_time = rusage.ru_utime.tv_sec * 1000000UL + rusage.ru_utime.tv_usec / 1000UL; - sys_time = rusage.ru_stime.tv_sec * 1000000UL + rusage.ru_stime.tv_usec / 1000UL; + user_time = rusage.ru_utime.tv_sec * 1000000000UL + rusage.ru_utime.tv_usec; + sys_time = rusage.ru_stime.tv_sec * 1000000000UL + rusage.ru_stime.tv_usec; - page_reclaims = static_cast(rusage.ru_minflt); + soft_page_faults = static_cast(rusage.ru_minflt); + hard_page_faults = static_cast(rusage.ru_majflt); voluntary_context_switches = static_cast(rusage.ru_nvcsw); involuntary_context_switches = static_cast(rusage.ru_nivcsw); } - static void incrementProfileEvents(const RusageCounters & cur, const RusageCounters & prev) + static RusageCounters zeros(UInt64 real_time_ = getCurrentTimeNanoseconds()) { - ProfileEvents::increment(ProfileEvents::RealTimeMicroseconds, cur.real_time - prev.real_time); - ProfileEvents::increment(ProfileEvents::RusageUserTimeMicroseconds, cur.user_time - prev.user_time); - ProfileEvents::increment(ProfileEvents::RusageSystemTimeMicroseconds, cur.sys_time - prev.sys_time); - ProfileEvents::increment(ProfileEvents::RusagePageReclaims, cur.page_reclaims - prev.page_reclaims); - ProfileEvents::increment(ProfileEvents::RusagePageVoluntaryContextSwitches, cur.voluntary_context_switches - prev.voluntary_context_switches); - ProfileEvents::increment(ProfileEvents::RusagePageInvoluntaryContextSwitches, cur.involuntary_context_switches - prev.involuntary_context_switches); + RusageCounters res; + res.real_time = real_time_; + return res; } - static void updateProfileEvents(RusageCounters & last_counters) + static RusageCounters current(UInt64 real_time_ = getCurrentTimeNanoseconds()) + { + ::rusage rusage; + ::getrusage(RUSAGE_THREAD, &rusage); + return RusageCounters(rusage, real_time_); + } + + static void incrementProfileEvents(const RusageCounters & prev, const RusageCounters & curr, ProfileEvents::Counters & profile_events) + { + profile_events.increment(ProfileEvents::RealTimeMicroseconds, (curr.real_time - prev.real_time) / 1000U); + profile_events.increment(ProfileEvents::UserTimeMicroseconds, (curr.user_time - prev.user_time) / 1000U); + profile_events.increment(ProfileEvents::SystemTimeMicroseconds, (curr.sys_time - prev.sys_time) / 1000U); + + profile_events.increment(ProfileEvents::SoftPageFaults, curr.soft_page_faults - prev.soft_page_faults); + profile_events.increment(ProfileEvents::HardPageFaults, curr.hard_page_faults - prev.hard_page_faults); + } + + static void updateProfileEvents(RusageCounters & last_counters, ProfileEvents::Counters & profile_events) { auto current_counters = current(); - incrementProfileEvents(current_counters, last_counters); + incrementProfileEvents(last_counters, current_counters, profile_events); last_counters = current_counters; } }; @@ -169,18 +182,20 @@ struct TasksStatsCounters static TasksStatsCounters current(); - static void incrementProfileEvents(const TasksStatsCounters & curr, const TasksStatsCounters & prev) + static void incrementProfileEvents(const TasksStatsCounters & prev, const TasksStatsCounters & curr, ProfileEvents::Counters & profile_events) { - ProfileEvents::increment(ProfileEvents::OSReadBytes, curr.stat.read_bytes - prev.stat.read_bytes); - ProfileEvents::increment(ProfileEvents::OSWriteBytes, curr.stat.write_bytes - prev.stat.write_bytes); - ProfileEvents::increment(ProfileEvents::OSReadChars, curr.stat.read_char - prev.stat.read_char); - ProfileEvents::increment(ProfileEvents::OSWriteChars, curr.stat.write_char - prev.stat.write_char); + profile_events.increment(ProfileEvents::OSIOWaitMicroseconds, + safeDiff(prev.stat.blkio_delay_total, curr.stat.blkio_delay_total) / 1000U); + profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.stat.read_bytes, curr.stat.read_bytes)); + profile_events.increment(ProfileEvents::OSWriteBytes, safeDiff(prev.stat.write_bytes, curr.stat.write_bytes)); + profile_events.increment(ProfileEvents::OSReadChars, safeDiff(prev.stat.read_char, curr.stat.read_char)); + profile_events.increment(ProfileEvents::OSWriteChars, safeDiff(prev.stat.write_char, curr.stat.write_char)); } - static void updateProfileEvents(TasksStatsCounters & last_counters) + static void updateProfileEvents(TasksStatsCounters & last_counters, ProfileEvents::Counters & profile_events) { auto current_counters = current(); - incrementProfileEvents(current_counters, last_counters); + incrementProfileEvents(last_counters, current_counters, profile_events); last_counters = current_counters; } }; @@ -224,49 +239,66 @@ ThreadStatus::~ThreadStatus() } void ThreadStatus::attachQuery( - QueryStatus *parent_query_, - ProfileEvents::Counters *parent_counters, - MemoryTracker *parent_memory_tracker, + QueryStatus * parent_query_, + ProfileEvents::Counters * parent_counters, + MemoryTracker * parent_memory_tracker, bool check_detached) { std::lock_guard lock(mutex); - if (check_detached && is_active_query) - throw Exception("Query is already active", ErrorCodes::LOGICAL_ERROR); - - if (auto counters_parent = performance_counters.parent) - if (counters_parent != parent_counters) - LOG_WARNING(log, "Parent performance counters are already set, overwrite"); - - if (auto tracker_parent = memory_tracker.getParent()) - if (tracker_parent != parent_memory_tracker) - LOG_WARNING(log, "Parent memory tracker is already set, overwrite"); + if (is_active_query) + { + if (check_detached) + throw Exception("Can't attach query to the thread, it is already attached", ErrorCodes::LOGICAL_ERROR); + return; + } parent_query = parent_query_; - performance_counters.parent = parent_counters; + performance_counters.setParent(parent_counters); memory_tracker.setParent(parent_memory_tracker); memory_tracker.setDescription("(for thread)"); - /// Attach current thread to list of query threads - if (parent_query) + /// Try extract as many information as possible from ProcessList + if (auto query = getParentQuery()) { - std::lock_guard lock(parent_query->threads_mutex); - auto res = parent_query->thread_statuses.emplace(current_thread->poco_thread_number, current_thread); + /// Attach current thread to list of query threads + { + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ":" << __LINE__ << " " << query); + std::unique_lock lock(query->threads_mutex); - if (!res.second && res.first->second.get() != current_thread.get()) - throw Exception("Thread " + std::to_string(current_thread->poco_thread_number) + " is set twice", ErrorCodes::LOGICAL_ERROR); + if (query->thread_statuses.empty()) + query->master_thread = shared_from_this(); + + if (!query->thread_statuses.emplace(poco_thread_number, shared_from_this()).second) + throw Exception("Thread " + std::to_string(poco_thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); + + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ":" << __LINE__ << " " << query); + } + + query_context = query->tryGetQueryContext(); + + if (auto current_query_context = getQueryContext()) + { + log_to_query_thread_log = current_query_context->getSettingsRef().log_query_threads.value != 0; + + if (!getGlobalContext()) + global_context = ¤t_query_context->getGlobalContext(); + } } + query_start_time_nanoseconds = getCurrentTimeNanoseconds(); + query_start_time = time(nullptr); + /// First init of thread rusage counters, set real time to zero, other metrics remain as is if (is_first_query_of_the_thread) { - impl->last_rusage = RusageCounters::zeros(); + impl->last_rusage = RusageCounters::zeros(query_start_time_nanoseconds); impl->last_taskstats = TasksStatsCounters::zeros(); updatePerfomanceCountersImpl(); } else { - impl->last_rusage = RusageCounters::current(); + impl->last_rusage = RusageCounters::current(query_start_time_nanoseconds); impl->last_taskstats = TasksStatsCounters::current(); } @@ -277,8 +309,8 @@ void ThreadStatus::updatePerfomanceCountersImpl() { try { - RusageCounters::incrementProfileEvents(RusageCounters::current(), impl->last_rusage); - TasksStatsCounters::incrementProfileEvents(TasksStatsCounters::current(), impl->last_taskstats); + RusageCounters::updateProfileEvents(impl->last_rusage, performance_counters); + TasksStatsCounters::updateProfileEvents(impl->last_taskstats, performance_counters); } catch (...) { @@ -286,34 +318,89 @@ void ThreadStatus::updatePerfomanceCountersImpl() } } -void ThreadStatus::detachQuery() +void ThreadStatus::detachQuery(bool thread_exits) { - std::lock_guard lock(mutex); if (!is_active_query) return; updatePerfomanceCountersImpl(); - is_first_query_of_the_thread = false; - is_active_query = false; + try + { + if (log_to_query_thread_log) + if (auto global_context = getGlobalContext()) + if (auto thread_log = global_context->getQueryThreadLog()) + logToQueryThreadLog(*thread_log); + } + catch (...) + { + tryLogCurrentException(log); + } - /// Detach from parent - performance_counters.setParent(nullptr); - memory_tracker.setParent(nullptr); + { + std::lock_guard lock(mutex); + + /// Detach from parent + performance_counters.setParent(nullptr); + memory_tracker.setParent(nullptr); + query_context = nullptr; + + is_active_query = false; + is_first_query_of_the_thread = false; + is_active_thread = !thread_exits; + } +} + +void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) +{ + QueryThreadLogElement elem; + + elem.event_time = time(nullptr); + elem.query_start_time = query_start_time; + elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U; + + elem.read_rows = progress_in.rows.load(std::memory_order_relaxed); + elem.read_bytes = progress_in.bytes.load(std::memory_order_relaxed); + elem.written_rows = progress_out.rows.load(std::memory_order_relaxed); + elem.written_bytes = progress_out.bytes.load(std::memory_order_relaxed); + elem.memory_usage = std::max(0, memory_tracker.getPeak()); + + elem.thread_number = poco_thread_number; + elem.os_thread_id = os_thread_id; + + if (auto query = getParentQuery()) + { + if (query->master_thread) + { + elem.master_thread_number = query->master_thread->poco_thread_number; + elem.master_os_thread_id = query->master_thread->os_thread_id; + } + + elem.query = query->query; + elem.client_info = query->getClientInfo(); + } + + if (auto current_context = getQueryContext()) + { + if (current_context->getSettingsRef().log_profile_events) + { + /// NOTE: Here we are in the same thread, so we can make memcpy() + elem.profile_counters = std::make_shared(); + performance_counters.getPartiallyAtomicSnapshot(*elem.profile_counters); + } + } + + thread_log.add(elem); } void ThreadStatus::reset() { std::lock_guard lock(mutex); + parent_query = nullptr; + if (is_active_query) throw Exception("Query is still active", ErrorCodes::LOGICAL_ERROR); - - performance_counters.reset(); - memory_tracker.reset(); - - performance_counters.setParent(nullptr); - memory_tracker.setParent(nullptr); } } diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 7992627087e..6e56119d61b 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include #include #include @@ -14,59 +15,95 @@ namespace Poco namespace DB { +class Context; class QueryStatus; class ThreadStatus; class ScopeCurrentThread; +class QueryThreadLog; using ThreadStatusPtr = std::shared_ptr; extern thread_local ThreadStatusPtr current_thread; -class ThreadStatus +class ThreadStatus : public std::enable_shared_from_this { public: UInt32 poco_thread_number = 0; - QueryStatus * parent_query = nullptr; ProfileEvents::Counters performance_counters; MemoryTracker memory_tracker; - int os_thread_id = -1; - std::mutex mutex; + Int32 os_thread_id = -1; + + Progress progress_in; + Progress progress_out; public: - /// A constructor static ThreadStatusPtr create(); - /// Reset all references and metrics + /// Called by master thread when the query finishes void reset(); + QueryStatus * getParentQuery() + { + return parent_query.load(std::memory_order_relaxed); + } + + Context * getGlobalContext() + { + return global_context.load(std::memory_order_relaxed); + } + + Context * getQueryContext() + { + return query_context.load(std::memory_order_relaxed); + } + ~ThreadStatus(); protected: ThreadStatus(); + void attachQuery( - QueryStatus *parent_query_, - ProfileEvents::Counters *parent_counters, + QueryStatus * parent_query_, + ProfileEvents::Counters * parent_counters, MemoryTracker *parent_memory_tracker, bool check_detached = true); - void detachQuery(); + + void detachQuery(bool thread_exits = false); + + void logToQueryThreadLog(QueryThreadLog & thread_log); + void updatePerfomanceCountersImpl(); - bool is_active_query = false; - bool is_active_thread = false; - bool is_first_query_of_the_thread = true; - Poco::Logger * log; + std::mutex mutex; + std::atomic is_active_query{false}; + bool is_active_thread{false}; + bool is_first_query_of_the_thread{true}; + + UInt64 query_start_time_nanoseconds{0}; + time_t query_start_time{0}; + + std::atomic parent_query{nullptr}; + /// Use it only from current thread + std::atomic query_context{nullptr}; + /// Is set once + std::atomic global_context{nullptr}; + + bool log_to_query_thread_log = false; + + Poco::Logger * log = nullptr; - friend class CurrentThreadScope; friend class CurrentThread; friend struct TasksStatsCounters; struct Impl; std::unique_ptr impl; - Impl & getImpl() { return *impl; } + +public: + class CurrentThreadScope; }; } diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index 0d7b1075c9d..3533111f624 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -99,8 +99,7 @@ protected: /// If there were no calculations yet, calculate the first block synchronously if (!started) { - auto main_thread = CurrentThread::get(); - calculate(main_thread); + calculate(); started = true; } else /// If the calculations are already in progress - wait for the result @@ -124,15 +123,17 @@ protected: void next() { ready.reset(); - pool.schedule([this, main_thread=CurrentThread::get()] () { calculate(main_thread); }); + pool.schedule([this, main_thread=CurrentThread::get()] (){ + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + calculate(); + }); } /// Calculations that can be performed in a separate thread - void calculate(ThreadStatusPtr main_thread) + void calculate() { CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); try { diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 25ca02a0398..ba94c7abe86 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace ProfileEvents @@ -301,11 +301,8 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0) { /// Do not count sleeps in throttlers - if (current_thread) - { - UInt64 throttler_sleeps_ms = current_thread->performance_counters[ProfileEvents::ThrottlerSleepMicroseconds]; - total_elapsed -= static_cast(throttler_sleeps_ms) / 1000000.0; - } + double throttler_sleeps_seconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds] / 1000000.0; + total_elapsed = std::max(0.0, total_elapsed - throttler_sleeps_seconds); if (limits.min_execution_speed && progress.rows / total_elapsed < limits.min_execution_speed) throw Exception("Query is executing too slow: " + toString(progress.rows / total_elapsed) diff --git a/dbms/src/IO/ReadBufferFromFileDescriptor.cpp b/dbms/src/IO/ReadBufferFromFileDescriptor.cpp index 6218697795f..19d5a641c86 100644 --- a/dbms/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/dbms/src/IO/ReadBufferFromFileDescriptor.cpp @@ -48,10 +48,7 @@ bool ReadBufferFromFileDescriptor::nextImpl() { ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorRead); - StopWatchRusage watch_ru; - std::optional watch; - if (profile_callback) - watch.emplace(clock_type); + Stopwatch watch(profile_callback ? clock_type : CLOCK_MONOTONIC); ssize_t res = 0; { @@ -70,15 +67,17 @@ bool ReadBufferFromFileDescriptor::nextImpl() if (res > 0) bytes_read += res; - watch_ru.stop(); - ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch_ru.elapsedMicroseconds()); + /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one and we will count cpu time of other thread + /// It is better to use taskstats::blkio_delay_total, but it is quite expensive to get it (TaskStatsInfoGetter has about 500K RPS) + watch.stop(); + ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds()); if (profile_callback) { ProfileInfo info; info.bytes_requested = internal_buffer.size(); info.bytes_read = res; - info.nanoseconds = watch->elapsed(); + info.nanoseconds = watch.elapsed(); profile_callback(info); } } diff --git a/dbms/src/IO/ReadBufferFromPocoSocket.cpp b/dbms/src/IO/ReadBufferFromPocoSocket.cpp index 88686b8b33c..bcaca18192d 100644 --- a/dbms/src/IO/ReadBufferFromPocoSocket.cpp +++ b/dbms/src/IO/ReadBufferFromPocoSocket.cpp @@ -27,7 +27,7 @@ namespace ErrorCodes bool ReadBufferFromPocoSocket::nextImpl() { ssize_t bytes_read = 0; - StopWatchRusage watch_ru; + Stopwatch watch; /// Add more details to exceptions. try @@ -50,7 +50,8 @@ bool ReadBufferFromPocoSocket::nextImpl() if (bytes_read < 0) throw NetException("Cannot read from socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_READ_FROM_SOCKET); - ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch_ru.elapsedMicroseconds()); + /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one + ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); if (bytes_read) working_buffer.resize(bytes_read); diff --git a/dbms/src/IO/WriteBufferFromFileDescriptor.cpp b/dbms/src/IO/WriteBufferFromFileDescriptor.cpp index 75f5aefef71..928b338ac31 100644 --- a/dbms/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/dbms/src/IO/WriteBufferFromFileDescriptor.cpp @@ -40,7 +40,7 @@ void WriteBufferFromFileDescriptor::nextImpl() if (!offset()) return; - StopWatchRusage watch_ru; + Stopwatch watch; size_t bytes_written = 0; while (bytes_written != offset()) @@ -63,7 +63,7 @@ void WriteBufferFromFileDescriptor::nextImpl() bytes_written += res; } - ProfileEvents::increment(ProfileEvents::DiskWriteElapsedMicroseconds, watch_ru.elapsedMicroseconds()); + ProfileEvents::increment(ProfileEvents::DiskWriteElapsedMicroseconds, watch.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteBytes, bytes_written); } diff --git a/dbms/src/IO/WriteBufferFromPocoSocket.cpp b/dbms/src/IO/WriteBufferFromPocoSocket.cpp index cefee566055..ef95767d5e1 100644 --- a/dbms/src/IO/WriteBufferFromPocoSocket.cpp +++ b/dbms/src/IO/WriteBufferFromPocoSocket.cpp @@ -29,7 +29,7 @@ void WriteBufferFromPocoSocket::nextImpl() if (!offset()) return; - StopWatchRusage watch_ru; + Stopwatch watch; size_t bytes_written = 0; while (bytes_written < offset()) @@ -58,9 +58,9 @@ void WriteBufferFromPocoSocket::nextImpl() throw NetException("Cannot write to socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_WRITE_TO_SOCKET); bytes_written += res; - - ProfileEvents::increment(ProfileEvents::NetworkSendElapsedMicroseconds, watch_ru.elapsedMicroseconds()); } + + ProfileEvents::increment(ProfileEvents::NetworkSendElapsedMicroseconds, watch.elapsedMicroseconds()); } WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 89e5764c33d..2333d392576 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -1513,22 +1514,36 @@ QueryLog * Context::getQueryLog() if (!global_context) throw Exception("Logical error: no global context for query log", ErrorCodes::LOGICAL_ERROR); - auto & config = getConfigRef(); - - String database = config.getString("query_log.database", "system"); - String table = config.getString("query_log.table", "query_log"); - String partition_by = config.getString("query_log.partition_by", "toYYYYMM(event_date)"); - size_t flush_interval_milliseconds = config.getUInt64("query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS); - - String engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024"; - - system_logs->query_log = std::make_unique(*global_context, database, table, engine, flush_interval_milliseconds); + system_logs->query_log = createDefaultSystemLog(*global_context, "system", "query_log", getConfigRef(), "query_log"); } return system_logs->query_log.get(); } +QueryThreadLog * Context::getQueryThreadLog() +{ + auto lock = getLock(); + + if (!system_logs) + return nullptr; + + if (!system_logs->query_thread_log) + { + if (shared->shutdown_called) + throw Exception("Logical error: query log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR); + + if (!global_context) + throw Exception("Logical error: no global context for query thread log", ErrorCodes::LOGICAL_ERROR); + + system_logs->query_thread_log = createDefaultSystemLog( + *global_context, "system", "query_thread_log", getConfigRef(), "query_thread_log"); + } + + return system_logs->query_thread_log.get(); +} + + PartLog * Context::getPartLog(const String & part_database) { auto lock = getLock(); @@ -1557,13 +1572,7 @@ PartLog * Context::getPartLog(const String & part_database) if (!global_context) throw Exception("Logical error: no global context for part log", ErrorCodes::LOGICAL_ERROR); - String table = config.getString("part_log.table", "part_log"); - String partition_by = config.getString("query_log.partition_by", "toYYYYMM(event_date)"); - size_t flush_interval_milliseconds = config.getUInt64("part_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS); - - String engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024"; - - system_logs->part_log = std::make_unique(*global_context, database, table, engine, flush_interval_milliseconds); + system_logs->part_log = createDefaultSystemLog(*global_context, "system", "part_log", getConfigRef(), "part_log"); } return system_logs->part_log.get(); @@ -1742,7 +1751,6 @@ std::shared_ptr Context::getActionLocksManager() return shared->action_locks_manager; } - SessionCleaner::~SessionCleaner() { try diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index cbe4123615f..bc7cd720028 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -52,6 +52,7 @@ class Macros; struct Progress; class Clusters; class QueryLog; +class QueryThreadLog; class PartLog; struct MergeTreeSettings; class IDatabase; @@ -351,6 +352,7 @@ public: /// Nullptr if the query log is not ready for this moment. QueryLog * getQueryLog(); + QueryThreadLog * getQueryThreadLog(); /// Returns an object used to log opertaions with parts if it possible. /// Provide table name to make required cheks. diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 5467637f960..826ed3e4d26 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -66,7 +66,7 @@ void ExternalLoader::init(bool throw_on_error) { /// During synchronous loading of external dictionaries at moment of query execution, /// we should not use per query memory limit. - auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerBlocker(); + auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock(); reloadAndUpdate(throw_on_error); } diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 125df365dda..1c08228fd68 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -168,8 +168,8 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as /// Attach master thread CurrentThread::attachQuery(&*process_it); - current_thread->memory_tracker.setOrRaiseLimit(settings.max_memory_usage); - current_thread->memory_tracker.setDescription("(for thread)"); + CurrentThread::getMemoryTracker().setOrRaiseLimit(settings.max_memory_usage); + CurrentThread::getMemoryTracker().setDescription("(for thread)"); if (!user_process_list.user_throttler) { @@ -199,8 +199,7 @@ ProcessListEntry::~ProcessListEntry() { CurrentThread::detachQuery(); - std::lock_guard lock(it->threads_mutex); - + std::shared_lock lock(it->threads_mutex); for (auto & elem : it->thread_statuses) { auto & thread_status = elem.second; @@ -286,6 +285,10 @@ QueryStatus::QueryStatus( memory_tracker.setFaultProbability(memory_tracker_fault_probability); } +QueryStatus::~QueryStatus() +{ + LOG_DEBUG(&Poco::Logger::get("QueryStatus"), __PRETTY_FUNCTION__ << ":" << __LINE__); +} void QueryStatus::setQueryStreams(const BlockIO & io) { @@ -335,7 +338,7 @@ bool QueryStatus::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStream void QueryStatus::setUserProcessList(ProcessListForUser * user_process_list_) { user_process_list = user_process_list_; - performance_counters.parent = &user_process_list->user_performance_counters; + performance_counters.setParent(&user_process_list->user_performance_counters); memory_tracker.setParent(&user_process_list->user_memory_tracker); } @@ -413,7 +416,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even if (get_thread_list) { - std::lock_guard lock(threads_mutex); + std::shared_lock lock(threads_mutex); res.thread_numbers.reserve(thread_statuses.size()); for (auto & thread_status_elem : thread_statuses) diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index 2519d1264e7..df1db61b498 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -16,11 +16,11 @@ #include #include #include +#include #include #include #include #include -#include "ThreadPerformanceProfile.h" namespace CurrentMetrics @@ -83,6 +83,8 @@ protected: String query; ClientInfo client_info; + + /// Is set once when init Context * query_context = nullptr; Stopwatch watch; @@ -99,8 +101,9 @@ protected: mutable std::shared_mutex threads_mutex; /// Key is Poco's thread_id - using QueryThreadStatuses = std::map; + using QueryThreadStatuses = std::map; QueryThreadStatuses thread_statuses; + ThreadStatusPtr master_thread; CurrentMetrics::Increment num_queries_increment{CurrentMetrics::Query}; @@ -138,10 +141,7 @@ public: double memory_tracker_fault_probability, QueryPriorities::Handle && priority_handle_); - ~QueryStatus() - { - // TODO: master thread should be reset - } + ~QueryStatus(); const ClientInfo & getClientInfo() const { @@ -162,6 +162,7 @@ public: bool updateProgressIn(const Progress & value) { + CurrentThread::updateProgressIn(value); progress_in.incrementPiecewiseAtomically(value); if (priority_handle) @@ -172,12 +173,15 @@ public: bool updateProgressOut(const Progress & value) { + CurrentThread::updateProgressOut(value); progress_out.incrementPiecewiseAtomically(value); + return !is_killed.load(std::memory_order_relaxed); } QueryStatusInfo getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const; + Context * tryGetQueryContext() { return query_context; } const Context * tryGetQueryContext() const { return query_context; } /// Copies pointers to in/out streams diff --git a/dbms/src/Interpreters/QueryLog.cpp b/dbms/src/Interpreters/QueryLog.cpp index 8385843fcdf..eccb8c472f6 100644 --- a/dbms/src/Interpreters/QueryLog.cpp +++ b/dbms/src/Interpreters/QueryLog.cpp @@ -21,58 +21,48 @@ Block QueryLogElement::createBlock() { return { - {ColumnUInt8::create(), std::make_shared(), "type"}, - {ColumnUInt16::create(), std::make_shared(), "event_date"}, - {ColumnUInt32::create(), std::make_shared(), "event_time"}, - {ColumnUInt32::create(), std::make_shared(), "query_start_time"}, - {ColumnUInt64::create(), std::make_shared(), "query_duration_ms"}, + {std::make_shared(), "type"}, + {std::make_shared(), "event_date"}, + {std::make_shared(), "event_time"}, + {std::make_shared(), "query_start_time"}, + {std::make_shared(), "query_duration_ms"}, - {ColumnUInt64::create(), std::make_shared(), "read_rows"}, - {ColumnUInt64::create(), std::make_shared(), "read_bytes"}, + {std::make_shared(), "read_rows"}, + {std::make_shared(), "read_bytes"}, + {std::make_shared(), "written_rows"}, + {std::make_shared(), "written_bytes"}, + {std::make_shared(), "result_rows"}, + {std::make_shared(), "result_bytes"}, + {std::make_shared(), "memory_usage"}, - {ColumnUInt64::create(), std::make_shared(), "written_rows"}, - {ColumnUInt64::create(), std::make_shared(), "written_bytes"}, + {std::make_shared(), "query"}, + {std::make_shared(), "exception"}, + {std::make_shared(), "stack_trace"}, - {ColumnUInt64::create(), std::make_shared(), "result_rows"}, - {ColumnUInt64::create(), std::make_shared(), "result_bytes"}, + {std::make_shared(), "is_initial_query"}, + {std::make_shared(), "user"}, + {std::make_shared(), "query_id"}, + {std::make_shared(16), "address"}, + {std::make_shared(), "port"}, + {std::make_shared(), "initial_user"}, + {std::make_shared(), "initial_query_id"}, + {std::make_shared(16), "initial_address"}, + {std::make_shared(), "initial_port"}, + {std::make_shared(), "interface"}, + {std::make_shared(), "os_user"}, + {std::make_shared(), "client_hostname"}, + {std::make_shared(), "client_name"}, + {std::make_shared(), "client_revision"}, + {std::make_shared(), "http_method"}, + {std::make_shared(), "http_user_agent"}, + {std::make_shared(), "quota_key"}, + {std::make_shared(), "revision"}, - {ColumnUInt64::create(), std::make_shared(), "memory_usage"}, - - {ColumnString::create(), std::make_shared(), "query"}, - {ColumnString::create(), std::make_shared(), "exception"}, - {ColumnString::create(), std::make_shared(), "stack_trace"}, - - {ColumnUInt8::create(), std::make_shared(), "is_initial_query"}, - - {ColumnString::create(), std::make_shared(), "user"}, - {ColumnString::create(), std::make_shared(), "query_id"}, - {ColumnFixedString::create(16), std::make_shared(16), "address"}, - {ColumnUInt16::create(), std::make_shared(), "port"}, - - {ColumnString::create(), std::make_shared(), "initial_user"}, - {ColumnString::create(), std::make_shared(), "initial_query_id"}, - {ColumnFixedString::create(16), std::make_shared(16), "initial_address"}, - {ColumnUInt16::create(), std::make_shared(), "initial_port"}, - - {ColumnUInt8::create(), std::make_shared(), "interface"}, - - {ColumnString::create(), std::make_shared(), "os_user"}, - {ColumnString::create(), std::make_shared(), "client_hostname"}, - {ColumnString::create(), std::make_shared(), "client_name"}, - {ColumnUInt32::create(), std::make_shared(), "client_revision"}, - - {ColumnUInt8::create(), std::make_shared(), "http_method"}, - {ColumnString::create(), std::make_shared(), "http_user_agent"}, - - {ColumnString::create(), std::make_shared(), "quota_key"}, - - {ColumnUInt32::create(), std::make_shared(), "revision"}, - - { std::make_shared(std::make_shared()), "thread_numbers" }, - { std::make_shared(std::make_shared()), "ProfileEvents.Names" }, - { std::make_shared(std::make_shared()), "ProfileEvents.Values" }, - { std::make_shared(std::make_shared()), "Settings.Names" }, - { std::make_shared(std::make_shared()), "Settings.Values" } + {std::make_shared(std::make_shared()), "thread_numbers"}, + {std::make_shared(std::make_shared()), "ProfileEvents.Names"}, + {std::make_shared(std::make_shared()), "ProfileEvents.Values"}, + {std::make_shared(std::make_shared()), "Settings.Names"}, + {std::make_shared(std::make_shared()), "Settings.Values"} }; } @@ -114,10 +104,8 @@ void QueryLogElement::appendToBlock(Block & block) const columns[i++]->insert(UInt64(read_rows)); columns[i++]->insert(UInt64(read_bytes)); - columns[i++]->insert(UInt64(written_rows)); columns[i++]->insert(UInt64(written_bytes)); - columns[i++]->insert(UInt64(result_rows)); columns[i++]->insert(UInt64(result_bytes)); @@ -127,29 +115,7 @@ void QueryLogElement::appendToBlock(Block & block) const columns[i++]->insertData(exception.data(), exception.size()); columns[i++]->insertData(stack_trace.data(), stack_trace.size()); - columns[i++]->insert(UInt64(client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY)); - - columns[i++]->insert(client_info.current_user); - columns[i++]->insert(client_info.current_query_id); - columns[i++]->insertData(IPv6ToBinary(client_info.current_address.host()).data(), 16); - columns[i++]->insert(UInt64(client_info.current_address.port())); - - columns[i++]->insert(client_info.initial_user); - columns[i++]->insert(client_info.initial_query_id); - columns[i++]->insertData(IPv6ToBinary(client_info.initial_address.host()).data(), 16); - columns[i++]->insert(UInt64(client_info.initial_address.port())); - - columns[i++]->insert(UInt64(client_info.interface)); - - columns[i++]->insert(client_info.os_user); - columns[i++]->insert(client_info.client_hostname); - columns[i++]->insert(client_info.client_name); - columns[i++]->insert(UInt64(client_info.client_revision)); - - columns[i++]->insert(UInt64(client_info.http_method)); - columns[i++]->insert(client_info.http_user_agent); - - columns[i++]->insert(client_info.quota_key); + appendClientInfo(client_info, columns, i); columns[i++]->insert(UInt64(ClickHouseRevision::get())); @@ -188,4 +154,31 @@ void QueryLogElement::appendToBlock(Block & block) const block.setColumns(std::move(columns)); } +void QueryLogElement::appendClientInfo(const ClientInfo & client_info, MutableColumns & columns, size_t & i) +{ + columns[i++]->insert(UInt64(client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY)); + + columns[i++]->insert(client_info.current_user); + columns[i++]->insert(client_info.current_query_id); + columns[i++]->insertData(IPv6ToBinary(client_info.current_address.host()).data(), 16); + columns[i++]->insert(UInt64(client_info.current_address.port())); + + columns[i++]->insert(client_info.initial_user); + columns[i++]->insert(client_info.initial_query_id); + columns[i++]->insertData(IPv6ToBinary(client_info.initial_address.host()).data(), 16); + columns[i++]->insert(UInt64(client_info.initial_address.port())); + + columns[i++]->insert(UInt64(client_info.interface)); + + columns[i++]->insert(client_info.os_user); + columns[i++]->insert(client_info.client_hostname); + columns[i++]->insert(client_info.client_name); + columns[i++]->insert(UInt64(client_info.client_revision)); + + columns[i++]->insert(UInt64(client_info.http_method)); + columns[i++]->insert(client_info.http_user_agent); + + columns[i++]->insert(client_info.quota_key); +} + } diff --git a/dbms/src/Interpreters/QueryLog.h b/dbms/src/Interpreters/QueryLog.h index 5f4a09e6736..a187f0adcb4 100644 --- a/dbms/src/Interpreters/QueryLog.h +++ b/dbms/src/Interpreters/QueryLog.h @@ -61,6 +61,8 @@ struct QueryLogElement static Block createBlock(); void appendToBlock(Block & block) const; + + static void appendClientInfo(const ClientInfo & client_info, MutableColumns & columns, size_t & i); }; diff --git a/dbms/src/Interpreters/QueryThreadLog.cpp b/dbms/src/Interpreters/QueryThreadLog.cpp new file mode 100644 index 00000000000..4e56770ada8 --- /dev/null +++ b/dbms/src/Interpreters/QueryThreadLog.cpp @@ -0,0 +1,108 @@ +#include "QueryThreadLog.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +Block QueryThreadLogElement::createBlock() +{ + return + { + {std::make_shared(), "event_date"}, + {std::make_shared(), "event_time"}, + {std::make_shared(), "query_start_time"}, + {std::make_shared(), "query_duration_ms"}, + + {std::make_shared(), "read_rows"}, + {std::make_shared(), "read_bytes"}, + {std::make_shared(), "written_rows"}, + {std::make_shared(), "written_bytes"}, + {std::make_shared(), "memory_usage"}, + + {std::make_shared(), "thread_number"}, + {std::make_shared(), "os_thread_id"}, + {std::make_shared(), "master_thread_number"}, + {std::make_shared(), "master_os_thread_id"}, + {std::make_shared(), "query"}, + + {std::make_shared(), "is_initial_query"}, + {std::make_shared(), "user"}, + {std::make_shared(), "query_id"}, + {std::make_shared(16), "address"}, + {std::make_shared(), "port"}, + {std::make_shared(), "initial_user"}, + {std::make_shared(), "initial_query_id"}, + {std::make_shared(16), "initial_address"}, + {std::make_shared(), "initial_port"}, + {std::make_shared(), "interface"}, + {std::make_shared(), "os_user"}, + {std::make_shared(), "client_hostname"}, + {std::make_shared(), "client_name"}, + {std::make_shared(), "client_revision"}, + {std::make_shared(), "http_method"}, + {std::make_shared(), "http_user_agent"}, + {std::make_shared(), "quota_key"}, + + {std::make_shared(), "revision"}, + + {std::make_shared(std::make_shared()), "ProfileEvents.Names"}, + {std::make_shared(std::make_shared()), "ProfileEvents.Values"} + }; +} + +void QueryThreadLogElement::appendToBlock(Block & block) const +{ + MutableColumns columns = block.mutateColumns(); + + size_t i = 0; + + columns[i++]->insert(UInt64(DateLUT::instance().toDayNum(event_time))); + columns[i++]->insert(UInt64(event_time)); + columns[i++]->insert(UInt64(query_start_time)); + columns[i++]->insert(UInt64(query_duration_ms)); + + columns[i++]->insert(UInt64(read_rows)); + columns[i++]->insert(UInt64(read_bytes)); + columns[i++]->insert(UInt64(written_rows)); + columns[i++]->insert(UInt64(written_bytes)); + + columns[i++]->insert(UInt64(memory_usage)); + + columns[i++]->insert(UInt64(thread_number)); + columns[i++]->insert(Int64(os_thread_id)); + columns[i++]->insert(UInt64(master_thread_number)); + columns[i++]->insert(Int64(master_os_thread_id)); + + columns[i++]->insertData(query.data(), query.size()); + + QueryLogElement::appendClientInfo(client_info, columns, i); + + columns[i++]->insert(UInt64(ClickHouseRevision::get())); + + if (profile_counters) + { + auto column_names = columns[i++].get(); + auto column_values = columns[i++].get(); + profile_counters->dumpToArrayColumns(column_names, column_values, true); + } + else + { + columns[i++]->insertDefault(); + columns[i++]->insertDefault(); + } +} + +} diff --git a/dbms/src/Interpreters/QueryThreadLog.h b/dbms/src/Interpreters/QueryThreadLog.h new file mode 100644 index 00000000000..8b5c2fc598b --- /dev/null +++ b/dbms/src/Interpreters/QueryThreadLog.h @@ -0,0 +1,53 @@ +#pragma once + +#include + + +namespace DB +{ + +struct QueryThreadLogElement +{ + time_t event_time{}; + /// When query was attached to current thread + time_t query_start_time{}; + /// Real time spent by the thread to execute the query + UInt64 query_duration_ms{}; + + /// The data fetched from DB in current thread to execute the query + UInt64 read_rows{}; + UInt64 read_bytes{}; + + /// The data written to DB + UInt64 written_rows{}; + UInt64 written_bytes{}; + + UInt64 memory_usage{}; + + UInt32 thread_number{}; + Int32 os_thread_id{}; + UInt32 master_thread_number{}; + Int32 master_os_thread_id{}; + + String query; + ClientInfo client_info; + + std::shared_ptr profile_counters; + + static std::string name() { return "QueryThreadLog"; } + + static Block createBlock(); + void appendToBlock(Block & block) const; +}; + + +class QueryThreadLog : public SystemLog +{ + using SystemLog::SystemLog; +}; + + +} + + + diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index f8d39244e70..c4dcffee3d9 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -256,8 +256,9 @@ struct Settings M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.") \ M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.") \ M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \ - M(SettingBool, log_profile_events, true, "Log query settings into the query_log.") \ - M(SettingBool, log_query_settings, true, "Log query performance statistics into the query_log.") + M(SettingBool, log_profile_events, true, "Log query settings into the query_log and query_thread_log.") \ + M(SettingBool, log_query_settings, true, "Log query performance statistics into the query_log.") \ + M(SettingBool, log_query_threads, true, "Log query threads into system.query_log table.") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index a68a55c79b2..bc5edaba14a 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index f4a45b45237..584146b0de5 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -53,6 +53,7 @@ namespace DB class Context; class QueryLog; +class QueryThreadLog; class PartLog; @@ -62,8 +63,9 @@ struct SystemLogs { ~SystemLogs(); - std::unique_ptr query_log; /// Used to log queries. - std::unique_ptr part_log; /// Used to log operations with parts + std::unique_ptr query_log; /// Used to log queries. + std::unique_ptr query_thread_log; /// Used to log query threads. + std::unique_ptr part_log; /// Used to log operations with parts }; @@ -72,6 +74,8 @@ class SystemLog : private boost::noncopyable { public: + using Self = SystemLog; + /** Parameter: table name where to write log. * If table is not exists, then it get created with specified engine. * If it already exists, then its structure is checked to be compatible with structure of log record. @@ -346,5 +350,24 @@ void SystemLog::prepareTable() is_prepared = true; } +/// Creates a system log with MergeTree engines using parameters from config +template +std::unique_ptr createDefaultSystemLog( + Context & context_, + const String & default_database_name, + const String & default_table_name, + Poco::Util::AbstractConfiguration & config, + const String & config_prefix) +{ + String database = config.getString(config_prefix + ".database", default_database_name); + String table = config.getString(config_prefix + ".table", default_table_name); + String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); + String engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024"; + + size_t flush_interval_milliseconds = config.getUInt64("query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS); + + return std::make_unique(context_, database, table, engine, flush_interval_milliseconds); +} + } diff --git a/dbms/src/Interpreters/ThreadPerformanceProfile.h b/dbms/src/Interpreters/ThreadPerformanceProfile.h deleted file mode 100644 index b467f7f21b5..00000000000 --- a/dbms/src/Interpreters/ThreadPerformanceProfile.h +++ /dev/null @@ -1,23 +0,0 @@ -#pragma once - -#include -#include -#include - -#include -#include - -#include - -#include -#include -#include - - - -namespace DB -{ - - - -} diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 84852061b32..96098dd4116 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -262,7 +262,7 @@ void MergeTreeReader::Stream::loadMarks() auto load = [&]() -> MarkCache::MappedPtr { /// Memory for marks must not be accounted as memory usage for query, because they are stored in shared cache. - auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerBlocker(); + auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock(); size_t file_size = Poco::File(path).getSize(); size_t expected_file_size = sizeof(MarkInCompressedFile) * marks_count; diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 3977ce62d68..7c404bfbb3c 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -438,7 +438,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm * And otherwise it will look like excessively growing memory consumption in context of query. * (observed in long INSERT SELECTs) */ - auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerBlocker(); + auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock(); /// Write index. The index contains Primary Key value for each `index_granularity` row. for (size_t i = index_offset; i < rows; i += storage.index_granularity) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index a3b2cd41a64..7b8dc0ce344 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -177,7 +177,7 @@ static void appendBlock(const Block & from, Block & to) try { /// Avoid "memory limit exceeded" exceptions during rollback. - auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerBlocker(); + auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock(); for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no) { diff --git a/dbms/src/Storages/System/StorageSystemProcesses.cpp b/dbms/src/Storages/System/StorageSystemProcesses.cpp index 9a9f39208bf..48060ff6ee8 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.cpp +++ b/dbms/src/Storages/System/StorageSystemProcesses.cpp @@ -62,10 +62,10 @@ StorageSystemProcesses::StorageSystemProcesses(const std::string & name_) virtual_columns = ColumnsWithTypeAndName{ { std::make_shared(std::make_shared()), "thread_numbers" }, - { std::make_shared(std::make_shared()), "ProfileEvents.Names" }, - { std::make_shared(std::make_shared()), "ProfileEvents.Values" }, - { std::make_shared(std::make_shared()), "Settings.Names" }, - { std::make_shared(std::make_shared()), "Settings.Values" } + { std::make_shared(std::make_shared()), "ProfileEvents_Names" }, + { std::make_shared(std::make_shared()), "ProfileEvents_Values" }, + { std::make_shared(std::make_shared()), "Settings_Names" }, + { std::make_shared(std::make_shared()), "Settings_Values" } }; } From 549a6944d90c73199bddd0a1ee3e5383d69e0e4d Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Fri, 1 Jun 2018 14:58:17 +0300 Subject: [PATCH 070/315] Better AsynchronousBlockInputStream in sync mode. [#CLICKHOUSE-2910] --- dbms/src/Common/ThreadStatus.cpp | 13 ++-- dbms/src/Common/ThreadStatus.h | 3 +- dbms/src/Common/setThreadName.cpp | 27 +++++++- dbms/src/Common/setThreadName.h | 3 + .../AsynchronousBlockInputStream.h | 61 +------------------ dbms/src/Interpreters/QueryThreadLog.cpp | 2 + dbms/src/Interpreters/QueryThreadLog.h | 1 + 7 files changed, 43 insertions(+), 67 deletions(-) diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 275b4c7720c..cc173528a75 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -280,6 +280,7 @@ void ThreadStatus::attachQuery( if (auto current_query_context = getQueryContext()) { log_to_query_thread_log = current_query_context->getSettingsRef().log_query_threads.value != 0; + log_profile_events = current_query_context->getSettingsRef().log_profile_events.value != 0; if (!getGlobalContext()) global_context = ¤t_query_context->getGlobalContext(); @@ -365,6 +366,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) elem.written_bytes = progress_out.bytes.load(std::memory_order_relaxed); elem.memory_usage = std::max(0, memory_tracker.getPeak()); + elem.thread_name = getThreadName(); elem.thread_number = poco_thread_number; elem.os_thread_id = os_thread_id; @@ -380,14 +382,11 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) elem.client_info = query->getClientInfo(); } - if (auto current_context = getQueryContext()) + if (log_profile_events) { - if (current_context->getSettingsRef().log_profile_events) - { - /// NOTE: Here we are in the same thread, so we can make memcpy() - elem.profile_counters = std::make_shared(); - performance_counters.getPartiallyAtomicSnapshot(*elem.profile_counters); - } + /// NOTE: Here we are in the same thread, so we can make memcpy() + elem.profile_counters = std::make_shared(); + performance_counters.getPartiallyAtomicSnapshot(*elem.profile_counters); } thread_log.add(elem); diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 6e56119d61b..95ff050725d 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -92,7 +92,8 @@ protected: /// Is set once std::atomic global_context{nullptr}; - bool log_to_query_thread_log = false; + bool log_to_query_thread_log = true; + bool log_profile_events = true; Poco::Logger * log = nullptr; diff --git a/dbms/src/Common/setThreadName.cpp b/dbms/src/Common/setThreadName.cpp index 1a33c330027..a151b1952f1 100644 --- a/dbms/src/Common/setThreadName.cpp +++ b/dbms/src/Common/setThreadName.cpp @@ -6,10 +6,19 @@ #else #include #endif +#include #include #include +namespace DB +{ +namespace ErrorCodes +{ + extern const int PTHREAD_ERROR; +} +} + void setThreadName(const char * name) { @@ -22,5 +31,21 @@ void setThreadName(const char * name) #else if (0 != prctl(PR_SET_NAME, name, 0, 0, 0)) #endif - DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME...)"); + DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME, ...)"); +} + +std::string getThreadName() +{ + std::string name(16, '\0'); + +#if defined(__FreeBSD__) || defined(__APPLE__) + if (pthread_get_name_np(pthread_self(), name.data(), name.size()); + throw DB::Exception("Cannot get thread name with pthread_get_name_np()", DB::ErrorCodes::PTHREAD_ERROR); +#else + if (0 != prctl(PR_GET_NAME, name.data(), 0, 0, 0)) +#endif + DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)"); + + name.resize(strlen(name.data())); + return name; } diff --git a/dbms/src/Common/setThreadName.h b/dbms/src/Common/setThreadName.h index dc6af7336e0..cdcb6b46914 100644 --- a/dbms/src/Common/setThreadName.h +++ b/dbms/src/Common/setThreadName.h @@ -1,7 +1,10 @@ #pragma once +#include /** Sets the thread name (maximum length is 15 bytes), * which will be visible in ps, gdb, /proc, * for convenience of observation and debugging. */ void setThreadName(const char * name); + +std::string getThreadName(); diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index 3533111f624..c790deb49c2 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -7,7 +7,6 @@ #include #include #include -#include #include @@ -93,66 +92,12 @@ protected: Block block; std::exception_ptr exception; + Block readImpl() override; - Block readImpl() override - { - /// If there were no calculations yet, calculate the first block synchronously - if (!started) - { - calculate(); - started = true; - } - else /// If the calculations are already in progress - wait for the result - pool.wait(); - - if (exception) - std::rethrow_exception(exception); - - Block res = block; - if (!res) - return res; - - /// Start the next block calculation - block.clear(); - next(); - - return res; - } - - - void next() - { - ready.reset(); - pool.schedule([this, main_thread=CurrentThread::get()] (){ - CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); - calculate(); - }); - } - + void next(); /// Calculations that can be performed in a separate thread - void calculate() - { - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - - try - { - if (first) - { - first = false; - setThreadName("AsyncBlockInput"); - children.back()->readPrefix(); - } - - block = children.back()->read(); - } - catch (...) - { - exception = std::current_exception(); - } - - ready.set(); - } + void calculate(); }; } diff --git a/dbms/src/Interpreters/QueryThreadLog.cpp b/dbms/src/Interpreters/QueryThreadLog.cpp index 4e56770ada8..dc290917f56 100644 --- a/dbms/src/Interpreters/QueryThreadLog.cpp +++ b/dbms/src/Interpreters/QueryThreadLog.cpp @@ -32,6 +32,7 @@ Block QueryThreadLogElement::createBlock() {std::make_shared(), "written_bytes"}, {std::make_shared(), "memory_usage"}, + {std::make_shared(), "thread_name"}, {std::make_shared(), "thread_number"}, {std::make_shared(), "os_thread_id"}, {std::make_shared(), "master_thread_number"}, @@ -81,6 +82,7 @@ void QueryThreadLogElement::appendToBlock(Block & block) const columns[i++]->insert(UInt64(memory_usage)); + columns[i++]->insertData(thread_name.data(), thread_name.size()); columns[i++]->insert(UInt64(thread_number)); columns[i++]->insert(Int64(os_thread_id)); columns[i++]->insert(UInt64(master_thread_number)); diff --git a/dbms/src/Interpreters/QueryThreadLog.h b/dbms/src/Interpreters/QueryThreadLog.h index 8b5c2fc598b..3552f15c623 100644 --- a/dbms/src/Interpreters/QueryThreadLog.h +++ b/dbms/src/Interpreters/QueryThreadLog.h @@ -24,6 +24,7 @@ struct QueryThreadLogElement UInt64 memory_usage{}; + String thread_name; UInt32 thread_number{}; Int32 os_thread_id{}; UInt32 master_thread_number{}; From 2b179b560b9962b76b4e383867d4bcc484e54949 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Fri, 1 Jun 2018 18:32:27 +0300 Subject: [PATCH 071/315] Initialize external tables after enqueue query. [#CLICKHOUSE-2910] --- dbms/src/Common/ExternalTable.h | 11 ++---- dbms/src/Common/ThreadStatus.cpp | 7 +--- dbms/src/Interpreters/Context.cpp | 18 +++++++++ dbms/src/Interpreters/Context.h | 10 +++++ dbms/src/Interpreters/executeQuery.cpp | 3 ++ dbms/src/Server/HTTPHandler.cpp | 45 +++++++++++++++-------- dbms/src/Server/TCPHandler.cpp | 17 ++++++--- dbms/src/Storages/MergeTree/MergeList.cpp | 4 +- 8 files changed, 79 insertions(+), 36 deletions(-) diff --git a/dbms/src/Common/ExternalTable.h b/dbms/src/Common/ExternalTable.h index c894b09c99b..8a4c2aaed06 100644 --- a/dbms/src/Common/ExternalTable.h +++ b/dbms/src/Common/ExternalTable.h @@ -50,9 +50,8 @@ public: { initReadBuffer(); initSampleBlock(); - ExternalTableData res = std::make_pair(std::make_shared(context.getInputFormat( - format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE)), name); - return res; + auto input = context.getInputFormat(format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE); + return std::make_pair(std::make_shared(input), name); } protected: @@ -169,9 +168,8 @@ public: class ExternalTablesHandler : public Poco::Net::PartHandler, BaseExternalTable { public: - std::vector names; - ExternalTablesHandler(Context & context_, Poco::Net::NameValueCollection params_) : context(context_), params(params_) { } + ExternalTablesHandler(Context & context_, const Poco::Net::NameValueCollection & params_) : context(context_), params(params_) { } void handlePart(const Poco::Net::MessageHeader & header, std::istream & stream) { @@ -211,14 +209,13 @@ public: data.first->readSuffix(); output->writeSuffix(); - names.push_back(name); /// We are ready to receive the next file, for this we clear all the information received clean(); } private: Context & context; - Poco::Net::NameValueCollection params; + const Poco::Net::NameValueCollection & params; }; diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index cc173528a75..fc5615586c7 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -101,7 +101,7 @@ static UInt64 getCurrentTimeNanoseconds(clockid_t clock_type = CLOCK_MONOTONIC) { struct timespec ts; clock_gettime(clock_type, &ts); - return ts.tv_sec * 1000000ULL + ts.tv_nsec / 1000UL; + return ts.tv_sec * 1000000000ULL + ts.tv_nsec; } @@ -254,7 +254,7 @@ void ThreadStatus::attachQuery( } parent_query = parent_query_; - performance_counters.setParent(parent_counters); + performance_counters.setParent(parent_counters ? parent_counters : &ProfileEvents::global_counters); memory_tracker.setParent(parent_memory_tracker); memory_tracker.setDescription("(for thread)"); @@ -263,7 +263,6 @@ void ThreadStatus::attachQuery( { /// Attach current thread to list of query threads { - LOG_DEBUG(log, __PRETTY_FUNCTION__ << ":" << __LINE__ << " " << query); std::unique_lock lock(query->threads_mutex); if (query->thread_statuses.empty()) @@ -271,8 +270,6 @@ void ThreadStatus::attachQuery( if (!query->thread_statuses.emplace(poco_thread_number, shared_from_this()).second) throw Exception("Thread " + std::to_string(poco_thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); - - LOG_DEBUG(log, __PRETTY_FUNCTION__ << ":" << __LINE__ << " " << query); } query_context = query->tryGetQueryContext(); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 2333d392576..6ac18cfbc0b 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -1751,6 +1751,24 @@ std::shared_ptr Context::getActionLocksManager() return shared->action_locks_manager; } +void Context::setExternalTablesInitializer(ExternalTablesInitializer && initializer) +{ + if (external_tables_initializer_callback) + throw Exception("External tables initializer is already set", ErrorCodes::LOGICAL_ERROR); + + external_tables_initializer_callback = std::move(initializer); +} + +void Context::initializeExternalTablesIfSet() +{ + if (external_tables_initializer_callback) + { + external_tables_initializer_callback(*this); + /// Reset callback + external_tables_initializer_callback = {}; + } +} + SessionCleaner::~SessionCleaner() { try diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index bc7cd720028..36b2e1e2905 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -33,6 +33,7 @@ namespace DB { struct ContextShared; +class Context; class IRuntimeComponentsFactory; class QuotaForIntervals; class EmbeddedDictionaries; @@ -85,6 +86,9 @@ using Dependencies = std::vector; using TableAndCreateAST = std::pair; using TableAndCreateASTs = std::map; +/// Callback for external tables initializer +using ExternalTablesInitializer = std::function; + /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). @@ -100,6 +104,7 @@ private: std::shared_ptr runtime_components_factory; ClientInfo client_info; + ExternalTablesInitializer external_tables_initializer_callback; std::shared_ptr quota; /// Current quota. By default - empty quota, that have no limits. String current_database; @@ -161,6 +166,11 @@ public: /// Compute and set actual user settings, client_info.current_user should be set void calculateUserSettings(); + /// We have to copy external tables in executeQuery(). Therefore, set callback for it. Must set once. + void setExternalTablesInitializer(ExternalTablesInitializer && initializer); + /// This method is called in executeQuery() and will call the external tables initializer. + void initializeExternalTablesIfSet(); + ClientInfo & getClientInfo() { return client_info; }; const ClientInfo & getClientInfo() const { return client_info; }; diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 587bd6498eb..fc48e1a8ba6 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -202,6 +202,9 @@ static std::tuple executeQueryImpl( context.setProcessListElement(&process_list_entry->get()); } + /// Load external tables if they were provided + context.initializeExternalTablesIfSet(); + auto interpreter = InterpreterFactory::get(ast, context, stage); res = interpreter->execute(); diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index fba60c5c12b..06fc1559357 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -421,22 +421,40 @@ void HTTPHandler::processQuery( std::unique_ptr in; + static const NameSet reserved_param_names{"query", "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check" + }; + + Names reserved_param_suffixes; + + auto param_could_be_skipped = [&] (const String & name) + { + if (reserved_param_names.count(name)) + return true; + + for (const String & suffix : reserved_param_suffixes) + { + if (endsWith(name, suffix)) + return true; + } + + return false; + }; + /// Support for "external data for query processing". if (startsWith(request.getContentType().data(), "multipart/form-data")) { in = std::move(in_param); - ExternalTablesHandler handler(context, params); - params.load(request, istr, handler); + context.setExternalTablesInitializer([¶ms, &request, &istr] (Context & context_query) { + ExternalTablesHandler handler(context_query, params); + params.load(request, istr, handler); + }); - /// Erase unneeded parameters to avoid confusing them later with context settings or query - /// parameters. - for (const auto & it : handler.names) - { - params.erase(it + "_format"); - params.erase(it + "_types"); - params.erase(it + "_structure"); - } + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + reserved_param_suffixes.emplace_back("_format"); + reserved_param_suffixes.emplace_back("_types"); + reserved_param_suffixes.emplace_back("_structure"); } else in = std::make_unique(*in_param, *in_post_maybe_compressed); @@ -463,11 +481,6 @@ void HTTPHandler::processQuery( auto readonly_before_query = settings.readonly; - NameSet reserved_param_names{"query", "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", - "buffer_size", "wait_end_of_query", - "session_id", "session_timeout", "session_check" - }; - for (auto it = params.begin(); it != params.end(); ++it) { if (it->first == "database") @@ -478,7 +491,7 @@ void HTTPHandler::processQuery( { context.setDefaultFormat(it->second); } - else if (reserved_param_names.find(it->first) != reserved_param_names.end()) + else if (param_could_be_skipped(it->first)) { } else diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 53ca6c8699f..020c8c03ec8 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -150,13 +150,18 @@ void TCPHandler::runImpl() if (!receivePacket()) continue; - /// Get blocks of temporary tables - readData(global_settings); + query_context.setExternalTablesInitializer([&global_settings, this] (Context & context) { + if (&context != &query_context) + throw Exception("Unexpected context in external tables initializer", ErrorCodes::LOGICAL_ERROR); - /// Reset the input stream, as we received an empty block while receiving external table data. - /// So, the stream has been marked as cancelled and we can't read from it anymore. - state.block_in.reset(); - state.maybe_compressed_in.reset(); /// For more accurate accounting by MemoryTracker. + /// Get blocks of temporary tables + readData(global_settings); + + /// Reset the input stream, as we received an empty block while receiving external table data. + /// So, the stream has been marked as cancelled and we can't read from it anymore. + state.block_in.reset(); + state.maybe_compressed_in.reset(); /// For more accurate accounting by MemoryTracker. + }); /// Processing Query state.io = executeQuery(state.query, query_context, false, state.stage); diff --git a/dbms/src/Storages/MergeTree/MergeList.cpp b/dbms/src/Storages/MergeTree/MergeList.cpp index 48c3a72da29..12f3fa08202 100644 --- a/dbms/src/Storages/MergeTree/MergeList.cpp +++ b/dbms/src/Storages/MergeTree/MergeList.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace CurrentMetrics @@ -22,7 +22,7 @@ MergeListElement::MergeListElement(const std::string & database, const std::stri source_part_names.emplace_back(source_part->name); /// Each merge is executed into separate background processing pool thread - background_thread_memory_tracker = current_thread ? ¤t_thread->memory_tracker : nullptr; + background_thread_memory_tracker = &CurrentThread::getMemoryTracker(); if (background_thread_memory_tracker) { memory_tracker.setMetric(CurrentMetrics::MemoryTrackingForMerges); From b46f2ad9468cc3ad6e14eded750b03e8f46a968d Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Fri, 1 Jun 2018 22:39:32 +0300 Subject: [PATCH 072/315] Simplified code via introducing states. [#CLICKHOUSE-2910] --- dbms/src/Common/CurrentThread.cpp | 41 ++++--- dbms/src/Common/CurrentThread.h | 8 +- dbms/src/Common/ProfileEvents.cpp | 8 +- dbms/src/Common/ProfileEvents.h | 20 ++-- dbms/src/Common/ThreadStatus.cpp | 161 +++++++++++++------------- dbms/src/Common/ThreadStatus.h | 45 +++---- dbms/src/Interpreters/ProcessList.cpp | 12 +- dbms/src/Server/HTTPHandler.cpp | 3 + dbms/src/Server/TCPHandler.cpp | 4 +- 9 files changed, 164 insertions(+), 138 deletions(-) diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 6f11ab93309..fb663f93cb2 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -22,13 +22,19 @@ static ThreadStatusPtr assertCurrentThread() if (!thread) throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR); - if (Poco::ThreadNumber::get() != thread->poco_thread_number) + if (Poco::ThreadNumber::get() != thread->thread_number) throw Exception("Current thread has different thread number", ErrorCodes::LOGICAL_ERROR); return thread; } +void CurrentThread::initializeQuery() +{ + assertCurrentThread(); + current_thread->initializeQuery(); +} + void CurrentThread::attachQuery(QueryStatus * parent_process) { assertCurrentThread(); @@ -68,11 +74,6 @@ void CurrentThread::detachQuery() current_thread->detachQuery(); } -bool CurrentThread::isAttachedToQuery() -{ - return current_thread->is_active_query; -} - ProfileEvents::Counters & CurrentThread::getProfileEvents() { return current_thread->performance_counters; @@ -95,26 +96,38 @@ void CurrentThread::updateProgressOut(const Progress & value) void CurrentThread::attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thread, bool check_detached) { - LOG_DEBUG(&Poco::Logger::get("CurrentThread"), __PRETTY_FUNCTION__ << ":" << __LINE__ << " " << "?" << " " << sibling_thread.get()); - if (sibling_thread == nullptr) throw Exception("Sibling thread was not initialized", ErrorCodes::LOGICAL_ERROR); assertCurrentThread(); + if (sibling_thread->getCurrentState() == ThreadStatus::ThreadState::QueryInitializing) + { + LOG_WARNING(current_thread->log, "An attempt to use initializing sibling thread detected." + << " Performance statistics for this thread will be inaccurate"); + } + QueryStatus * parent_query; ProfileEvents::Counters * parent_counters; MemoryTracker * parent_memory_tracker; { /// NOTE: It is almost the only place where ThreadStatus::mutex is required - /// In other cases ThreadStatus must be accessed from the current_thread + /// In other cases ThreadStatus must be accessed only from the current_thread std::lock_guard lock(sibling_thread->mutex); - parent_query = sibling_thread->getParentQuery(); - parent_counters = sibling_thread->performance_counters.getParent(); - parent_memory_tracker = sibling_thread->memory_tracker.getParent(); - } - LOG_DEBUG(&Poco::Logger::get("CurrentThread"), __PRETTY_FUNCTION__ << ":" << __LINE__ << " " << parent_query << " " << sibling_thread.get()); + parent_query = sibling_thread->parent_query; + if (parent_query) + { + parent_counters = &parent_query->performance_counters; + parent_memory_tracker = &parent_query->memory_tracker; + } + else + { + /// Fallback + parent_counters = sibling_thread->performance_counters.getParent(); + parent_memory_tracker = sibling_thread->memory_tracker.getParent(); + } + } current_thread->attachQuery(parent_query, parent_counters, parent_memory_tracker, check_detached); } diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index 95046a458ca..9f191be7d34 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -25,8 +25,11 @@ public: static ThreadStatusPtr get(); - /// You must call one of these methods when create a child thread: + /// Call when thread accepted connection (but haven't called executeQuery()) + /// Currently it is used only for debugging + static void initializeQuery(); + /// You must call one of these methods when create a child thread: /// Bundles the current thread with a query static void attachQuery(QueryStatus * parent_process); /// Bundles the current thread with a query bundled to the sibling thread @@ -34,8 +37,9 @@ public: /// Is useful for a ThreadPool tasks static void attachQueryFromSiblingThreadIfDetached(const ThreadStatusPtr & sibling_thread); + /// Makes system calls to update ProfileEvents derived from rusage and taskstats static void updatePerformanceCounters(); - static bool isAttachedToQuery(); + static ProfileEvents::Counters & getProfileEvents(); static MemoryTracker & getMemoryTracker(); static void updateProgressIn(const Progress & value); diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 882e6faa9ea..976f8976547 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -182,8 +182,8 @@ const Event Counters::num_counters = END; Counters::Counters(Level level, Counters * parent) - : parent(parent), - counters_holder(new Counter[num_counters] {}), + : counters_holder(new Counter[num_counters] {}), + parent(parent), level(level) { counters = counters_holder.get(); @@ -204,10 +204,12 @@ void Counters::reset() resetCounters(); } -void Counters::getPartiallyAtomicSnapshot(Counters & res) const +Counters Counters::getPartiallyAtomicSnapshot() const { + Counters res(Level::Snapshot, nullptr); for (Event i = 0; i < num_counters; ++i) res.counters[i].store(counters[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + return res; } const char * getDescription(Event event) diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index 31f269b422c..2f7ab5ea26c 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -22,26 +22,33 @@ namespace ProfileEvents using Event = size_t; using Count = size_t; using Counter = std::atomic; + class Counters; + + /// Counters - how many times each event happened + extern Counters global_counters; enum class Level { Global = 0, User, Process, - Thread + Thread, + Snapshot }; class Counters { Counter * counters = nullptr; - Counters * parent = nullptr; std::unique_ptr counters_holder; + /// Used to propagate increments + Counters * parent = nullptr; public: Level level = Level::Thread; - Counters(Level level = Level::Thread, Counters * parent = nullptr); + /// By default, any instance have to increment global counters + Counters(Level level = Level::Thread, Counters * parent = &global_counters); /// Global level static initializer Counters(Counter * allocated_counters) @@ -62,7 +69,7 @@ namespace ProfileEvents } while (current != nullptr); } - void getPartiallyAtomicSnapshot(Counters & res) const; + Counters getPartiallyAtomicSnapshot() const; /// Reset metrics and parent void reset(); @@ -88,11 +95,6 @@ namespace ProfileEvents static const Event num_counters; }; - - /// Counters - how many times each event happened. - extern Counters global_counters; - - /// Increment a counter for event. Thread-safe. void increment(Event event, Count amount = 1); diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index fc5615586c7..d05fc9c54b5 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -52,11 +52,7 @@ public: { try { - { - std::lock_guard lock(current_thread->mutex); - current_thread->is_active_thread = true; - } - LOG_DEBUG(current_thread->log, "Thread " << current_thread->poco_thread_number << " is started"); + LOG_DEBUG(current_thread->log, "Thread " << current_thread->thread_number << " is started"); } catch (...) { @@ -68,12 +64,8 @@ public: { try { - CurrentThread::detachQuery(); - LOG_DEBUG(current_thread->log, "Thread " << current_thread->poco_thread_number << " is exited"); - { - std::lock_guard lock(current_thread->mutex); - current_thread->is_active_thread = false; - } + current_thread->detachQuery(true); + LOG_DEBUG(current_thread->log, "Thread " << current_thread->thread_number << " is exited"); } catch (...) { @@ -218,14 +210,14 @@ TasksStatsCounters TasksStatsCounters::current() ThreadStatus::ThreadStatus() - : poco_thread_number(Poco::ThreadNumber::get()), + : thread_number(Poco::ThreadNumber::get()), performance_counters(ProfileEvents::Level::Thread), os_thread_id(TaskStatsInfoGetter::getCurrentTID()), log(&Poco::Logger::get("ThreadStatus")) { impl = std::make_unique(); - LOG_DEBUG(log, "Thread " << poco_thread_number << " created"); + LOG_DEBUG(log, "Thread " << thread_number << " created"); } ThreadStatusPtr ThreadStatus::create() @@ -235,7 +227,15 @@ ThreadStatusPtr ThreadStatus::create() ThreadStatus::~ThreadStatus() { - LOG_DEBUG(log, "Thread " << poco_thread_number << " destroyed in " << Poco::ThreadNumber::get()); + LOG_DEBUG(log, "Thread " << thread_number << " destroyed"); +} + +void ThreadStatus::initializeQuery() +{ + if (thread_state != ThreadState::QueryInitializing && thread_state != ThreadState::DetachedFromQuery) + throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); + + thread_state = ThreadState::QueryInitializing; } void ThreadStatus::attachQuery( @@ -244,83 +244,66 @@ void ThreadStatus::attachQuery( MemoryTracker * parent_memory_tracker, bool check_detached) { - std::lock_guard lock(mutex); - - if (is_active_query) + if (thread_state == ThreadState::AttachedToQuery) { if (check_detached) throw Exception("Can't attach query to the thread, it is already attached", ErrorCodes::LOGICAL_ERROR); return; } - parent_query = parent_query_; - performance_counters.setParent(parent_counters ? parent_counters : &ProfileEvents::global_counters); - memory_tracker.setParent(parent_memory_tracker); - memory_tracker.setDescription("(for thread)"); + if (thread_state != ThreadState::DetachedFromQuery && thread_state != ThreadState::QueryInitializing) + throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); + + { + std::lock_guard lock(mutex); + parent_query = parent_query_; + performance_counters.setParent(parent_counters); + memory_tracker.setParent(parent_memory_tracker); + memory_tracker.setDescription("(for thread)"); + } /// Try extract as many information as possible from ProcessList - if (auto query = getParentQuery()) + if (parent_query) { /// Attach current thread to list of query threads { - std::unique_lock lock(query->threads_mutex); + std::unique_lock lock(parent_query->threads_mutex); - if (query->thread_statuses.empty()) - query->master_thread = shared_from_this(); + if (parent_query->thread_statuses.empty()) + parent_query->master_thread = shared_from_this(); - if (!query->thread_statuses.emplace(poco_thread_number, shared_from_this()).second) - throw Exception("Thread " + std::to_string(poco_thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); + if (!parent_query->thread_statuses.emplace(thread_number, shared_from_this()).second) + throw Exception("Thread " + std::to_string(thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); } - query_context = query->tryGetQueryContext(); - - if (auto current_query_context = getQueryContext()) + query_context = parent_query->tryGetQueryContext(); + if (query_context) { - log_to_query_thread_log = current_query_context->getSettingsRef().log_query_threads.value != 0; - log_profile_events = current_query_context->getSettingsRef().log_profile_events.value != 0; + log_to_query_thread_log = query_context->getSettingsRef().log_query_threads.value != 0; + log_profile_events = query_context->getSettingsRef().log_profile_events.value != 0; if (!getGlobalContext()) - global_context = ¤t_query_context->getGlobalContext(); + global_context = &query_context->getGlobalContext(); } } + thread_state = ThreadState::AttachedToQuery; + query_start_time_nanoseconds = getCurrentTimeNanoseconds(); query_start_time = time(nullptr); - /// First init of thread rusage counters, set real time to zero, other metrics remain as is - if (is_first_query_of_the_thread) - { - impl->last_rusage = RusageCounters::zeros(query_start_time_nanoseconds); - impl->last_taskstats = TasksStatsCounters::zeros(); - updatePerfomanceCountersImpl(); - } - else - { - impl->last_rusage = RusageCounters::current(query_start_time_nanoseconds); - impl->last_taskstats = TasksStatsCounters::current(); - } - - is_active_query = true; -} - -void ThreadStatus::updatePerfomanceCountersImpl() -{ - try - { - RusageCounters::updateProfileEvents(impl->last_rusage, performance_counters); - TasksStatsCounters::updateProfileEvents(impl->last_taskstats, performance_counters); - } - catch (...) - { - tryLogCurrentException(log); - } + impl->last_rusage = RusageCounters::current(query_start_time_nanoseconds); + impl->last_taskstats = TasksStatsCounters::current(); } void ThreadStatus::detachQuery(bool thread_exits) { - if (!is_active_query) + if (thread_state == ThreadStatus::DetachedFromQuery) return; + if (thread_state != ThreadState::AttachedToQuery && thread_state != ThreadState::QueryInitializing) + throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); + updatePerfomanceCountersImpl(); try @@ -339,13 +322,27 @@ void ThreadStatus::detachQuery(bool thread_exits) std::lock_guard lock(mutex); /// Detach from parent - performance_counters.setParent(nullptr); + performance_counters.setParent(&ProfileEvents::global_counters); memory_tracker.setParent(nullptr); query_context = nullptr; + } - is_active_query = false; - is_first_query_of_the_thread = false; - is_active_thread = !thread_exits; + thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; + + log_to_query_thread_log = true; + log_profile_events = true; +} + +void ThreadStatus::updatePerfomanceCountersImpl() +{ + try + { + RusageCounters::updateProfileEvents(impl->last_rusage, performance_counters); + TasksStatsCounters::updateProfileEvents(impl->last_taskstats, performance_counters); + } + catch (...) + { + tryLogCurrentException(log); } } @@ -364,39 +361,43 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) elem.memory_usage = std::max(0, memory_tracker.getPeak()); elem.thread_name = getThreadName(); - elem.thread_number = poco_thread_number; + elem.thread_number = thread_number; elem.os_thread_id = os_thread_id; - if (auto query = getParentQuery()) + if (parent_query) { - if (query->master_thread) { - elem.master_thread_number = query->master_thread->poco_thread_number; - elem.master_os_thread_id = query->master_thread->os_thread_id; + std::shared_lock threads_mutex(parent_query->threads_mutex); + + if (parent_query->master_thread) + { + elem.master_thread_number = parent_query->master_thread->thread_number; + elem.master_os_thread_id = parent_query->master_thread->os_thread_id; + } } - elem.query = query->query; - elem.client_info = query->getClientInfo(); + elem.query = parent_query->query; + elem.client_info = parent_query->getClientInfo(); } if (log_profile_events) { /// NOTE: Here we are in the same thread, so we can make memcpy() - elem.profile_counters = std::make_shared(); - performance_counters.getPartiallyAtomicSnapshot(*elem.profile_counters); + elem.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); } thread_log.add(elem); } -void ThreadStatus::reset() +void ThreadStatus::clean() { - std::lock_guard lock(mutex); + { + std::lock_guard lock(mutex); + parent_query = nullptr; + } - parent_query = nullptr; - - if (is_active_query) - throw Exception("Query is still active", ErrorCodes::LOGICAL_ERROR); + if (thread_state != ThreadState::DetachedFromQuery && thread_state != ThreadState::Died) + throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 95ff050725d..93cde503f03 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -18,7 +18,6 @@ namespace DB class Context; class QueryStatus; class ThreadStatus; -class ScopeCurrentThread; class QueryThreadLog; using ThreadStatusPtr = std::shared_ptr; @@ -30,11 +29,13 @@ class ThreadStatus : public std::enable_shared_from_this { public: - UInt32 poco_thread_number = 0; + /// Poco's thread number (the same number is used in logs) + UInt32 thread_number = 0; ProfileEvents::Counters performance_counters; MemoryTracker memory_tracker; Int32 os_thread_id = -1; + /// Statistics of read and write rows/bytes Progress progress_in; Progress progress_out; @@ -43,11 +44,19 @@ public: static ThreadStatusPtr create(); /// Called by master thread when the query finishes - void reset(); + void clean(); - QueryStatus * getParentQuery() + enum ThreadState { - return parent_query.load(std::memory_order_relaxed); + DetachedFromQuery = 0, /// We just created thread or it is background thread + QueryInitializing, /// We accepted a connection, but haven't enqueued a query to ProcessList + AttachedToQuery, /// Thread executes enqueued query + Died, /// Thread does not exist + }; + + int getCurrentState() const + { + return thread_state.load(std::memory_order_relaxed); } Context * getGlobalContext() @@ -55,21 +64,18 @@ public: return global_context.load(std::memory_order_relaxed); } - Context * getQueryContext() - { - return query_context.load(std::memory_order_relaxed); - } - ~ThreadStatus(); protected: ThreadStatus(); + void initializeQuery(); + void attachQuery( QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, - MemoryTracker *parent_memory_tracker, + MemoryTracker * parent_memory_tracker, bool check_detached = true); void detachQuery(bool thread_exits = false); @@ -78,19 +84,18 @@ protected: void updatePerfomanceCountersImpl(); + std::atomic thread_state{ThreadState::DetachedFromQuery}; + std::mutex mutex; - std::atomic is_active_query{false}; - bool is_active_thread{false}; - bool is_first_query_of_the_thread{true}; + QueryStatus * parent_query = nullptr; - UInt64 query_start_time_nanoseconds{0}; - time_t query_start_time{0}; - - std::atomic parent_query{nullptr}; - /// Use it only from current thread - std::atomic query_context{nullptr}; /// Is set once std::atomic global_context{nullptr}; + /// Use it only from current thread + Context * query_context = nullptr; + + UInt64 query_start_time_nanoseconds = 0; + time_t query_start_time = 0; bool log_to_query_thread_log = true; bool log_profile_events = true; diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 1c08228fd68..b8595d604ac 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -201,10 +201,7 @@ ProcessListEntry::~ProcessListEntry() std::shared_lock lock(it->threads_mutex); for (auto & elem : it->thread_statuses) - { - auto & thread_status = elem.second; - thread_status->reset(); - }; + elem.second->clean(); } std::lock_guard lock(parent.mutex); @@ -420,14 +417,11 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even res.thread_numbers.reserve(thread_statuses.size()); for (auto & thread_status_elem : thread_statuses) - res.thread_numbers.emplace_back(thread_status_elem.second->poco_thread_number); + res.thread_numbers.emplace_back(thread_status_elem.second->thread_number); } if (get_profile_events) - { - res.profile_counters = std::make_shared(ProfileEvents::Level::Process); - performance_counters.getPartiallyAtomicSnapshot(*res.profile_counters); - } + res.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); if (get_settings && query_context) res.query_settings = std::make_shared(query_context->getSettingsRef()); diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index 06fc1559357..8be79a749dc 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -208,6 +209,8 @@ void HTTPHandler::processQuery( Poco::Net::HTTPServerResponse & response, Output & used_output) { + CurrentThread::initializeQuery(); + LOG_TRACE(log, "Request URI: " << request.getURI()); std::istream & istr = request.stream(); diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 020c8c03ec8..1e1a911fb19 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -3,7 +3,7 @@ #include #include - +#include #include #include @@ -150,6 +150,8 @@ void TCPHandler::runImpl() if (!receivePacket()) continue; + CurrentThread::initializeQuery(); + query_context.setExternalTablesInitializer([&global_settings, this] (Context & context) { if (&context != &query_context) throw Exception("Unexpected context in external tables initializer", ErrorCodes::LOGICAL_ERROR); From 89f2107c75e3e092cd6b006f4e64e4eed10ae629 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Mon, 4 Jun 2018 17:16:27 +0300 Subject: [PATCH 073/315] C++ify taskstats getter, periodic auto update of profile events. [#CLICKHOUSE-2910] --- dbms/src/Common/ProfileEvents.h | 2 +- dbms/src/Common/TaskStatsInfoGetter.cpp | 87 ++++++++++--------- dbms/src/Common/ThreadStatus.cpp | 52 ++++++----- dbms/src/Common/ThreadStatus.h | 14 ++- dbms/src/Core/Protocol.h | 2 + .../IProfilingBlockInputStream.cpp | 29 ++++--- .../DataStreams/IProfilingBlockInputStream.h | 2 + 7 files changed, 107 insertions(+), 81 deletions(-) diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index 2f7ab5ea26c..ed69c8d78ac 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -86,7 +86,7 @@ namespace ProfileEvents parent = parent_; } - /// Reset metrics + /// Set all counters to zero void resetCounters(); /// Dumps profile events to two column Array(String) and Array(UInt64) diff --git a/dbms/src/Common/TaskStatsInfoGetter.cpp b/dbms/src/Common/TaskStatsInfoGetter.cpp index 71bdfab32e6..408ca8c0a28 100644 --- a/dbms/src/Common/TaskStatsInfoGetter.cpp +++ b/dbms/src/Common/TaskStatsInfoGetter.cpp @@ -17,6 +17,7 @@ #include /// Based on: https://github.com/Tomas-M/iotop/tree/master/src +/// More info: https://www.kernel.org/doc/Documentation/accounting/taskstats.txt /* * Generic macros for dealing with netlink sockets. Might be duplicated @@ -44,10 +45,10 @@ namespace static size_t constexpr MAX_MSG_SIZE = 1024; -struct msgtemplate +struct NetlinkMessage { - struct nlmsghdr n; - struct genlmsghdr g; + ::nlmsghdr n; + ::genlmsghdr g; char buf[MAX_MSG_SIZE]; }; @@ -57,14 +58,9 @@ struct msgtemplate int send_cmd(int sock_fd, __u16 nlmsg_type, __u32 nlmsg_pid, __u8 genl_cmd, __u16 nla_type, - void *nla_data, int nla_len) + void * nla_data, int nla_len) noexcept { - struct nlattr *na; - struct sockaddr_nl nladdr; - int r, buflen; - char *buf; - - msgtemplate msg; + NetlinkMessage msg; memset(&msg, 0, sizeof(msg)); msg.n.nlmsg_len = NLMSG_LENGTH(GENL_HDRLEN); @@ -75,20 +71,27 @@ int send_cmd(int sock_fd, __u16 nlmsg_type, __u32 nlmsg_pid, msg.g.cmd = genl_cmd; msg.g.version = 0x1; - na = (struct nlattr *) GENLMSG_DATA(&msg); + ::nlattr * na = static_cast<::nlattr *>(GENLMSG_DATA(&msg)); na->nla_type = nla_type; na->nla_len = nla_len + 1 + NLA_HDRLEN; memcpy(NLA_DATA(na), nla_data, nla_len); msg.n.nlmsg_len += NLMSG_ALIGN(na->nla_len); - buf = (char *) &msg; - buflen = msg.n.nlmsg_len ; + char * buf = reinterpret_cast(&msg); + ssize_t buflen = msg.n.nlmsg_len; + + ::sockaddr_nl nladdr; memset(&nladdr, 0, sizeof(nladdr)); nladdr.nl_family = AF_NETLINK; - while ((r = sendto(sock_fd, buf, buflen, 0, (struct sockaddr *) &nladdr, - sizeof(nladdr))) < buflen) + + while (true) { + ssize_t r = ::sendto(sock_fd, buf, buflen, 0, reinterpret_cast<::sockaddr *>(&nladdr), sizeof(nladdr)); + + if (r >= buflen) + break; + if (r > 0) { buf += r; @@ -97,55 +100,52 @@ int send_cmd(int sock_fd, __u16 nlmsg_type, __u32 nlmsg_pid, else if (errno != EAGAIN) return -1; } + return 0; } -int get_family_id(int nl_sock_fd) +int get_family_id(int nl_sock_fd) noexcept { - static char name[256]; - struct { - struct nlmsghdr n; - struct genlmsghdr g; + ::nlmsghdr n; + ::genlmsghdr g; char buf[256]; } ans; - int id = 0; - struct nlattr *na; - int rep_len; + static char name[] = TASKSTATS_GENL_NAME; - strcpy(name, TASKSTATS_GENL_NAME); if (send_cmd(nl_sock_fd, GENL_ID_CTRL, getpid(), CTRL_CMD_GETFAMILY, CTRL_ATTR_FAMILY_NAME, (void *) name, strlen(TASKSTATS_GENL_NAME) + 1)) return 0; - rep_len = recv(nl_sock_fd, &ans, sizeof(ans), 0); - if (ans.n.nlmsg_type == NLMSG_ERROR - || (rep_len < 0) || !NLMSG_OK((&ans.n), rep_len)) + int id = 0; + ssize_t rep_len = ::recv(nl_sock_fd, &ans, sizeof(ans), 0); + if (ans.n.nlmsg_type == NLMSG_ERROR || (rep_len < 0) || !NLMSG_OK((&ans.n), rep_len)) return 0; - na = (struct nlattr *) GENLMSG_DATA(&ans); - na = (struct nlattr *) ((char *) na + NLA_ALIGN(na->nla_len)); + ::nlattr * na; + na = static_cast<::nlattr *>(GENLMSG_DATA(&ans)); + na = reinterpret_cast<::nlattr *>((char *) na + NLA_ALIGN(na->nla_len)); if (na->nla_type == CTRL_ATTR_FAMILY_ID) - id = *(__u16 *) NLA_DATA(na); + id = *static_cast<__u16 *>(NLA_DATA(na)); return id; } -bool get_taskstats(int nl_sock_fd, int nl_family_id, pid_t xxxid, struct taskstats & out_stats, Exception * out_exception = nullptr) +bool get_taskstats(int nl_sock_fd, int nl_family_id, pid_t xxxid, ::taskstats & out_stats, Exception * out_exception = nullptr) { if (send_cmd(nl_sock_fd, nl_family_id, xxxid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &xxxid, sizeof(pid_t))) throwFromErrno("Can't send a Netlink command"); - msgtemplate msg; - int rv = recv(nl_sock_fd, &msg, sizeof(msg), 0); + NetlinkMessage msg; + int rv = ::recv(nl_sock_fd, &msg, sizeof(msg), 0); if (msg.n.nlmsg_type == NLMSG_ERROR || !NLMSG_OK((&msg.n), rv)) { - struct nlmsgerr *err = static_cast(NLMSG_DATA(&msg)); + ::nlmsgerr * err = static_cast<::nlmsgerr *>(NLMSG_DATA(&msg)); Exception e("Can't get Netlink response, error=" + std::to_string(err->error), ErrorCodes::NETLINK_ERROR); if (out_exception) @@ -159,32 +159,33 @@ bool get_taskstats(int nl_sock_fd, int nl_family_id, pid_t xxxid, struct tasksta rv = GENLMSG_PAYLOAD(&msg.n); - struct nlattr *na = (struct nlattr *) GENLMSG_DATA(&msg); + ::nlattr * na = static_cast<::nlattr *>(GENLMSG_DATA(&msg)); int len = 0; while (len < rv) { len += NLA_ALIGN(na->nla_len); - if (na->nla_type == TASKSTATS_TYPE_AGGR_TGID - || na->nla_type == TASKSTATS_TYPE_AGGR_PID) + if (na->nla_type == TASKSTATS_TYPE_AGGR_TGID || na->nla_type == TASKSTATS_TYPE_AGGR_PID) { int aggr_len = NLA_PAYLOAD(na->nla_len); int len2 = 0; - na = (struct nlattr *) NLA_DATA(na); + na = static_cast<::nlattr *>(NLA_DATA(na)); while (len2 < aggr_len) { if (na->nla_type == TASKSTATS_TYPE_STATS) { - struct taskstats *ts = static_cast(NLA_DATA(na)); + ::taskstats *ts = static_cast<::taskstats *>(NLA_DATA(na)); out_stats = *ts; } + len2 += NLA_ALIGN(na->nla_len); - na = (struct nlattr *) ((char *) na + len2); + na = reinterpret_cast<::nlattr *>((char *) na + len2); } } - na = (struct nlattr *) ((char *) GENLMSG_DATA(&msg) + len); + + na = reinterpret_cast<::nlattr *>((char *) GENLMSG_DATA(&msg) + len); } return true; @@ -201,11 +202,11 @@ TaskStatsInfoGetter::TaskStatsInfoGetter() if (netlink_socket_fd < 0) throwFromErrno("Can't create PF_NETLINK socket"); - struct sockaddr_nl addr; + ::sockaddr_nl addr; memset(&addr, 0, sizeof(addr)); addr.nl_family = AF_NETLINK; - if (bind(netlink_socket_fd, (struct sockaddr *) &addr, sizeof(addr)) < 0) + if (::bind(netlink_socket_fd, reinterpret_cast<::sockaddr *>(&addr), sizeof(addr)) < 0) throwFromErrno("Can't bind PF_NETLINK socket"); netlink_family_id = get_family_id(netlink_socket_fd); diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index d05fc9c54b5..88eae23da44 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -52,7 +52,8 @@ public: { try { - LOG_DEBUG(current_thread->log, "Thread " << current_thread->thread_number << " is started"); + ThreadStatus & thread = *CurrentThread::get(); + LOG_DEBUG(thread.log, "Thread " << thread.thread_number << " started"); } catch (...) { @@ -64,8 +65,14 @@ public: { try { - current_thread->detachQuery(true); - LOG_DEBUG(current_thread->log, "Thread " << current_thread->thread_number << " is exited"); + ThreadStatus & thread = *CurrentThread::get(); + + if (thread.getCurrentState() != ThreadStatus::ThreadState::DetachedFromQuery) + thread.detachQuery(true); + else + thread.thread_state = ThreadStatus::ThreadState::Died; + + LOG_DEBUG(thread.log, "Thread " << thread.thread_number << " exited"); } catch (...) { @@ -192,30 +199,24 @@ struct TasksStatsCounters } }; -struct ThreadStatus::Impl -{ - RusageCounters last_rusage; - TasksStatsCounters last_taskstats; - TaskStatsInfoGetter info_getter; -}; - - TasksStatsCounters TasksStatsCounters::current() { TasksStatsCounters res; - current_thread->impl->info_getter.getStat(res.stat, current_thread->os_thread_id); + current_thread->taskstats_getter->getStat(res.stat); return res; } - ThreadStatus::ThreadStatus() - : thread_number(Poco::ThreadNumber::get()), - performance_counters(ProfileEvents::Level::Thread), - os_thread_id(TaskStatsInfoGetter::getCurrentTID()), + : performance_counters(ProfileEvents::Level::Thread), log(&Poco::Logger::get("ThreadStatus")) { - impl = std::make_unique(); + thread_number = Poco::ThreadNumber::get(); + os_thread_id = TaskStatsInfoGetter::getCurrentTID(); + + last_rusage = std::make_unique(); + last_taskstats = std::make_unique(); + taskstats_getter = std::make_unique(); LOG_DEBUG(log, "Thread " << thread_number << " created"); } @@ -291,16 +292,19 @@ void ThreadStatus::attachQuery( query_start_time_nanoseconds = getCurrentTimeNanoseconds(); query_start_time = time(nullptr); + ++queries_started; - impl->last_rusage = RusageCounters::current(query_start_time_nanoseconds); - impl->last_taskstats = TasksStatsCounters::current(); + /// Clear stats from previous query if a new query is started + /// TODO: make separate query_thread_performance_counters and thread_performance_counters + if (queries_started != 1) + performance_counters.resetCounters(); + + *last_rusage = RusageCounters::current(query_start_time_nanoseconds); + *last_taskstats = TasksStatsCounters::current(); } void ThreadStatus::detachQuery(bool thread_exits) { - if (thread_state == ThreadStatus::DetachedFromQuery) - return; - if (thread_state != ThreadState::AttachedToQuery && thread_state != ThreadState::QueryInitializing) throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); @@ -337,8 +341,8 @@ void ThreadStatus::updatePerfomanceCountersImpl() { try { - RusageCounters::updateProfileEvents(impl->last_rusage, performance_counters); - TasksStatsCounters::updateProfileEvents(impl->last_taskstats, performance_counters); + RusageCounters::updateProfileEvents(*last_rusage, performance_counters); + TasksStatsCounters::updateProfileEvents(*last_taskstats, performance_counters); } catch (...) { diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 93cde503f03..37742b7b6d0 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -19,6 +19,9 @@ class Context; class QueryStatus; class ThreadStatus; class QueryThreadLog; +struct TasksStatsCounters; +struct RusageCounters; +class TaskStatsInfoGetter; using ThreadStatusPtr = std::shared_ptr; @@ -31,9 +34,11 @@ public: /// Poco's thread number (the same number is used in logs) UInt32 thread_number = 0; + /// Linux's PID (or TGID) (the same id is shown by ps util) + Int32 os_thread_id = -1; + ProfileEvents::Counters performance_counters; MemoryTracker memory_tracker; - Int32 os_thread_id = -1; /// Statistics of read and write rows/bytes Progress progress_in; @@ -99,14 +104,17 @@ protected: bool log_to_query_thread_log = true; bool log_profile_events = true; + size_t queries_started = 0; Poco::Logger * log = nullptr; friend class CurrentThread; friend struct TasksStatsCounters; - struct Impl; - std::unique_ptr impl; + /// Use ptr to not add extra dependencies in header + std::unique_ptr last_rusage; + std::unique_ptr last_taskstats; + std::unique_ptr taskstats_getter; public: class CurrentThreadScope; diff --git a/dbms/src/Core/Protocol.h b/dbms/src/Core/Protocol.h index cd5456cca34..458744a3470 100644 --- a/dbms/src/Core/Protocol.h +++ b/dbms/src/Core/Protocol.h @@ -69,6 +69,7 @@ namespace Protocol Totals = 7, /// A block with totals (compressed or not). Extremes = 8, /// A block with minimums and maximums (compressed or not). TablesStatusResponse = 9, /// A response to TablesStatus request. + Log = 10 /// System logs of the query execution }; /// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10 @@ -97,6 +98,7 @@ namespace Protocol Cancel = 3, /// Cancel the query execution. Ping = 4, /// Check that connection to the server is alive. TablesStatusRequest = 5, /// Check status of tables on the server. + KeepAlive = 6 /// Keep the connection alive }; inline const char * toString(UInt64 packet) diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index ba94c7abe86..24bfc931896 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -294,18 +294,27 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) size_t total_rows = progress.total_rows; - if (limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0)) + constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds + UInt64 total_elapsed_microseconds = info.total_stopwatch.elapsedMicroseconds(); + + if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) { - double total_elapsed = info.total_stopwatch.elapsedSeconds(); + CurrentThread::updatePerformanceCounters(); + last_profile_events_update_time = total_elapsed_microseconds; + } - if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0) + if ((limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0)) + && (static_cast(total_elapsed_microseconds) > limits.timeout_before_checking_execution_speed.totalMicroseconds())) + { + /// Do not count sleeps in throttlers + UInt64 throttler_sleep_microseconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds]; + double elapsed_seconds = (throttler_sleep_microseconds > total_elapsed_microseconds) + ? 0.0 : (total_elapsed_microseconds - throttler_sleep_microseconds) / 1000000.0; + + if (elapsed_seconds > 0) { - /// Do not count sleeps in throttlers - double throttler_sleeps_seconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds] / 1000000.0; - total_elapsed = std::max(0.0, total_elapsed - throttler_sleeps_seconds); - - if (limits.min_execution_speed && progress.rows / total_elapsed < limits.min_execution_speed) - throw Exception("Query is executing too slow: " + toString(progress.rows / total_elapsed) + if (limits.min_execution_speed && progress.rows / elapsed_seconds < limits.min_execution_speed) + throw Exception("Query is executing too slow: " + toString(progress.rows / elapsed_seconds) + " rows/sec., minimum: " + toString(limits.min_execution_speed), ErrorCodes::TOO_SLOW); @@ -314,7 +323,7 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) /// If the predicted execution time is longer than `max_execution_time`. if (limits.max_execution_time != 0 && total_rows) { - double estimated_execution_time_seconds = total_elapsed * (static_cast(total_rows) / progress.rows); + double estimated_execution_time_seconds = elapsed_seconds * (static_cast(total_rows) / progress.rows); if (estimated_execution_time_seconds > limits.max_execution_time.totalSeconds()) throw Exception("Estimated query execution time (" + toString(estimated_execution_time_seconds) + " seconds)" diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.h b/dbms/src/DataStreams/IProfilingBlockInputStream.h index b3df3d02e91..ab0db8dd99d 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.h +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.h @@ -180,6 +180,8 @@ protected: std::atomic is_killed{false}; ProgressCallback progress_callback; QueryStatus * process_list_elem = nullptr; + /// According to total_stopwatch in microseconds + UInt64 last_profile_events_update_time = 0; /// Additional information that can be generated during the work process. From caa86bc59adf42c57354a88f0df38359cbb8059e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 May 2018 15:29:52 +0300 Subject: [PATCH 074/315] Fixed ColumnWithDictionary::serializeBinaryBulkWithMultipleStreams, added more comments. --- .../AggregateFunctionFactory.cpp | 2 +- dbms/src/Columns/ColumnUnique.h | 12 ++++++------ dbms/src/DataTypes/DataTypeWithDictionary.cpp | 15 +++++++++------ dbms/src/DataTypes/DataTypeWithDictionary.h | 2 +- 4 files changed, 17 insertions(+), 14 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 0bfbc5e7a58..90109ff04c5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -54,7 +54,7 @@ static DataTypes convertTypesWithDictionaryToNested(const DataTypes & types) res_types.push_back(type); } - return std::move(res_types); + return res_types; } AggregateFunctionPtr AggregateFunctionFactory::get( diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 4c61bbe9d47..8bc4a2a06a8 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -145,7 +145,7 @@ private: void buildIndex(); ColumnType * getRawColumnPtr() { return static_cast(column_holder->assumeMutable().get()); } const ColumnType * getRawColumnPtr() const { return static_cast(column_holder.get()); } - IndexType insert(const StringRefWrapper & ref, IndexType value); + IndexType insertIntoMap(const StringRefWrapper & ref, IndexType value); }; @@ -213,7 +213,7 @@ void ColumnUnique::buildIndex() } template -IndexType ColumnUnique::insert(const StringRefWrapper & ref, IndexType value) +IndexType ColumnUnique::insertIntoMap(const StringRefWrapper & ref, IndexType value) { if (!index) buildIndex(); @@ -242,7 +242,7 @@ size_t ColumnUnique::uniqueInsert(const Field & x) return getDefaultValueIndex(); column->insert(x); - auto pos = insert(StringRefWrapper(column, prev_size), prev_size); + auto pos = insertIntoMap(StringRefWrapper(column, prev_size), prev_size); if (pos != prev_size) column->popBack(1); @@ -272,7 +272,7 @@ size_t ColumnUnique::uniqueInsertData(const char * pos, s if (!index->has(StringRefWrapper(StringRef(pos, length)))) { column->insertData(pos, length); - return static_cast(insert(StringRefWrapper(StringRef(pos, length)), size)); + return static_cast(insertIntoMap(StringRefWrapper(StringRef(pos, length)), size)); } return size; @@ -299,7 +299,7 @@ size_t ColumnUnique::uniqueInsertDataWithTerminatingZero( return getDefaultValueIndex(); } - auto position = insert(StringRefWrapper(column, prev_size), prev_size); + auto position = insertIntoMap(StringRefWrapper(column, prev_size), prev_size); if (position != prev_size) column->popBack(1); @@ -319,7 +319,7 @@ size_t ColumnUnique::uniqueDeserializeAndInsertFromArena( return getDefaultValueIndex(); } - auto index_pos = insert(StringRefWrapper(column, prev_size), prev_size); + auto index_pos = insertIntoMap(StringRefWrapper(column, prev_size), prev_size); if (index_pos != prev_size) column->popBack(1); diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index a9a81660fa5..199904a98c7 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -71,17 +71,19 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); MutableColumnPtr sub_index; - if (limit == 0) - limit = column.size(); + size_t max_limit = column.size() - offset; + limit = limit ? std::min(limit, max_limit) : max_limit; path.push_back(Substream::DictionaryKeys); if (auto stream = getter(path)) { const auto & indexes = column_with_dictionary.getIndexesPtr(); const auto & keys = column_with_dictionary.getUnique()->getNestedColumn(); - sub_index = (*indexes->cut(offset, limit - offset)).mutate(); + sub_index = (*indexes->cut(offset, limit)).mutate(); ColumnPtr unique_indexes = makeSubIndex(*sub_index); + /// unique_indexes->index(sub_index) == indexes[offset:offset + limit] auto used_keys = keys->index(unique_indexes, 0); + /// (used_keys, sub_index) is ColumnWithDictionary for range [offset:offset + limit] UInt64 used_keys_size = used_keys->size(); writeIntBinary(used_keys_size, *stream); @@ -94,7 +96,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( if (!sub_index) throw Exception("Dictionary keys wasn't serialized", ErrorCodes::LOGICAL_ERROR); - indexes_type->serializeBinaryBulk(*sub_index, *stream, offset, limit); + indexes_type->serializeBinaryBulk(*sub_index, *stream, 0, limit); } } @@ -127,7 +129,8 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( auto index_col = indexes_type->createColumn(); indexes_type->deserializeBinaryBulk(*index_col, *stream, limit, 0); - column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(std::move(index_col), 0), 0, limit); + auto index_size = index_col->size(); + column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(std::move(index_col), 0), 0, index_size); } } @@ -226,7 +229,7 @@ MutableColumnPtr DataTypeWithDictionary::createColumn() const if (!column) throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR); - return std::move(column); + return column; } throw Exception("Unexpected dictionary type for DataTypeWithDictionary: " + type->getName(), diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index f8dc96e4669..a76ca22cd33 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -158,7 +158,7 @@ private: template MutableColumnPtr createColumnImpl() const; - friend class CreateColumnVector; + friend struct CreateColumnVector; }; } From d018f3ad60a3ff6c794e09d8f5c4ac2777802b21 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 May 2018 17:15:55 +0300 Subject: [PATCH 075/315] Per granula reading for ColumnWithDictionary in MergeTreeRangeReader. --- dbms/src/Storages/MergeTree/MergeTreeReader.cpp | 14 ++++++++++++-- dbms/src/Storages/MergeTree/MergeTreeReader.h | 1 + 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 7539a40a1a0..ebc32c902cd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -42,7 +42,7 @@ MergeTreeReader::MergeTreeReader(const String & path, clockid_t clock_type) : avg_value_size_hints(avg_value_size_hints), path(path), data_part(data_part), columns(columns) , uncompressed_cache(uncompressed_cache), mark_cache(mark_cache), save_marks_in_cache(save_marks_in_cache), storage(storage) - , all_mark_ranges(all_mark_ranges), aio_threshold(aio_threshold), max_read_buffer_size(max_read_buffer_size) + , all_mark_ranges(all_mark_ranges), aio_threshold(aio_threshold), max_read_buffer_size(max_read_buffer_size), index_granularity(storage.index_granularity) { try { @@ -383,7 +383,17 @@ void MergeTreeReader::readData( }; double & avg_value_size_hint = avg_value_size_hints[name]; - type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, avg_value_size_hint, true, {}); + if (column.withDictionary()) + { + for (size_t read_rows = 0; read_rows < max_rows_to_read; read_rows += index_granularity) + { + size_t rows_to_read = std::min(index_granularity, max_rows_to_read - read_rows); + type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, rows_to_read, avg_value_size_hint, true, {}); + continue_reading = true; + } + } + else + type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, avg_value_size_hint, true, {}); IDataType::updateAvgValueSizeHint(column, avg_value_size_hint); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index a4a6b4dc0f7..05ef06b7419 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -107,6 +107,7 @@ private: MarkRanges all_mark_ranges; size_t aio_threshold; size_t max_read_buffer_size; + size_t index_granularity; void addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); From ac54e52fb32e15f376cc128302c4df80e1decda1 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Wed, 6 Jun 2018 23:57:07 +0300 Subject: [PATCH 076/315] Add server logs passing to client. [#CLICKHOUSE-2910] --- dbms/src/Client/Connection.cpp | 46 +++++++- dbms/src/Client/Connection.h | 6 + dbms/src/Common/CurrentThread.cpp | 60 ++++++---- dbms/src/Common/CurrentThread.h | 5 + dbms/src/Common/ThreadStatus.cpp | 30 ++--- dbms/src/Common/ThreadStatus.h | 27 ++++- dbms/src/Core/Defines.h | 1 + dbms/src/Core/Protocol.h | 4 +- dbms/src/Core/SystemLogsQueue.cpp | 55 +++++++++ dbms/src/Core/SystemLogsQueue.h | 29 +++++ dbms/src/DataStreams/BlockIO.h | 13 ++- .../DataStreams/SystemLogsRowOutputStream.cpp | 63 ++++++++++ .../DataStreams/SystemLogsRowOutputStream.h | 36 ++++++ .../src/Interpreters/ClickHouseLogChannel.cpp | 45 +++++++ dbms/src/Interpreters/Settings.h | 3 +- dbms/src/Server/CMakeLists.txt | 5 +- dbms/src/Server/Client.cpp | 98 +++++++++++----- dbms/src/Server/ClusterCopier.cpp | 1 - dbms/src/Server/TCPHandler.cpp | 110 ++++++++++++++++-- dbms/src/Server/TCPHandler.h | 12 ++ .../include/daemon/OwnPatternFormatter.h | 1 + libs/libdaemon/src/BaseDaemon.cpp | 8 ++ libs/libdaemon/src/OwnPatternFormatter.cpp | 7 +- 23 files changed, 567 insertions(+), 98 deletions(-) create mode 100644 dbms/src/Core/SystemLogsQueue.cpp create mode 100644 dbms/src/Core/SystemLogsQueue.h create mode 100644 dbms/src/DataStreams/SystemLogsRowOutputStream.cpp create mode 100644 dbms/src/DataStreams/SystemLogsRowOutputStream.h create mode 100644 dbms/src/Interpreters/ClickHouseLogChannel.cpp diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 9cd6e29986d..3c8912e4d3d 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -354,6 +354,7 @@ void Connection::sendQuery( maybe_compressed_in.reset(); maybe_compressed_out.reset(); block_in.reset(); + block_logs_in.reset(); block_out.reset(); /// Send empty block which means end of data. @@ -489,13 +490,13 @@ bool Connection::hasReadBufferPendingData() const Connection::Packet Connection::receivePacket() { - //LOG_TRACE(log_wrapper.get(), "Receiving packet"); - try { Packet res; readVarUInt(res.type, *in); + // LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type)); + switch (res.type) { case Protocol::Server::Data: @@ -524,6 +525,10 @@ Connection::Packet Connection::receivePacket() res.block = receiveData(); return res; + case Protocol::Server::Log: + res.block = receiveLogData(); + return res; + case Protocol::Server::EndOfStream: return res; @@ -551,14 +556,26 @@ Block Connection::receiveData() //LOG_TRACE(log_wrapper.get(), "Receiving data"); initBlockInput(); + return receiveDataImpl(block_in); +} + +Block Connection::receiveLogData() +{ + initBlockLogsInput(); + return receiveDataImpl(block_logs_in); +} + + +Block Connection::receiveDataImpl(BlockInputStreamPtr & stream) +{ String external_table_name; readStringBinary(external_table_name, *in); size_t prev_bytes = in->count(); /// Read one block from network. - Block res = block_in->read(); + Block res = stream->read(); if (throttler) throttler->add(in->count() - prev_bytes); @@ -567,20 +584,39 @@ Block Connection::receiveData() } -void Connection::initBlockInput() +void Connection::initInputBuffers() { - if (!block_in) + if (!maybe_compressed_in) { if (compression == Protocol::Compression::Enable) maybe_compressed_in = std::make_shared(*in); else maybe_compressed_in = in; + } +} + +void Connection::initBlockInput() +{ + if (!block_in) + { + initInputBuffers(); block_in = std::make_shared(*maybe_compressed_in, server_revision); } } +void Connection::initBlockLogsInput() +{ + if (!block_logs_in) + { + initInputBuffers(); + /// Have to return superset of SystemLogsQueue::getSampleBlock() columns + block_logs_in = std::make_shared(*maybe_compressed_in, server_revision); + } +} + + void Connection::setDescription() { auto resolved_address = getResolvedAddress(); diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index ef941224c9f..6a8d0f71781 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -213,6 +213,7 @@ private: /// From where to read query execution result. std::shared_ptr maybe_compressed_in; BlockInputStreamPtr block_in; + BlockInputStreamPtr block_logs_in; /// Where to write data for INSERT. std::shared_ptr maybe_compressed_out; @@ -248,11 +249,16 @@ private: bool ping(); Block receiveData(); + Block receiveLogData(); + Block receiveDataImpl(BlockInputStreamPtr & stream); + std::unique_ptr receiveException(); Progress receiveProgress(); BlockStreamProfileInfo receiveProfileInfo(); + void initInputBuffers(); void initBlockInput(); + void initBlockLogsInput(); void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const; }; diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index fb663f93cb2..620a3a52738 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -15,34 +15,34 @@ namespace ErrorCodes } -static ThreadStatusPtr assertCurrentThread() +static ThreadStatusPtr getCurrentThreadImpl() { - ThreadStatusPtr thread = current_thread; - - if (!thread) + if (!current_thread || current_thread.use_count() <= 0) throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR); - if (Poco::ThreadNumber::get() != thread->thread_number) + if (Poco::ThreadNumber::get() != current_thread->thread_number) throw Exception("Current thread has different thread number", ErrorCodes::LOGICAL_ERROR); - return thread; + return current_thread; } void CurrentThread::initializeQuery() { - assertCurrentThread(); - current_thread->initializeQuery(); + getCurrentThreadImpl()->initializeQuery(); } void CurrentThread::attachQuery(QueryStatus * parent_process) { - assertCurrentThread(); + ThreadStatusPtr thread = getCurrentThreadImpl(); if (!parent_process) - current_thread->attachQuery(nullptr, nullptr, nullptr); + thread->attachQuery(nullptr, nullptr, nullptr, CurrentThread::getSystemLogsQueue()); else - current_thread->attachQuery(parent_process, &parent_process->performance_counters, &parent_process->memory_tracker); + { + thread->attachQuery( + parent_process, &parent_process->performance_counters, &parent_process->memory_tracker, CurrentThread::getSystemLogsQueue()); + } } @@ -58,20 +58,17 @@ void CurrentThread::attachQueryFromSiblingThreadIfDetached(const ThreadStatusPtr void CurrentThread::updatePerformanceCounters() { - assertCurrentThread(); - current_thread->updatePerfomanceCountersImpl(); + getCurrentThreadImpl()->updatePerfomanceCountersImpl(); } ThreadStatusPtr CurrentThread::get() { - assertCurrentThread(); - return current_thread; + return getCurrentThreadImpl(); } void CurrentThread::detachQuery() { - assertCurrentThread(); - current_thread->detachQuery(); + getCurrentThreadImpl()->detachQuery(); } ProfileEvents::Counters & CurrentThread::getProfileEvents() @@ -99,17 +96,18 @@ void CurrentThread::attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thr if (sibling_thread == nullptr) throw Exception("Sibling thread was not initialized", ErrorCodes::LOGICAL_ERROR); - assertCurrentThread(); + ThreadStatusPtr thread = getCurrentThreadImpl(); if (sibling_thread->getCurrentState() == ThreadStatus::ThreadState::QueryInitializing) { - LOG_WARNING(current_thread->log, "An attempt to use initializing sibling thread detected." - << " Performance statistics for this thread will be inaccurate"); + LOG_WARNING(thread->log, "An attempt to \'fork\' from initializing thread detected." + << " Performance statistics for this thread will be inaccurate"); } QueryStatus * parent_query; ProfileEvents::Counters * parent_counters; MemoryTracker * parent_memory_tracker; + SystemLogsQueueWeakPtr logs_queue_ptr; { /// NOTE: It is almost the only place where ThreadStatus::mutex is required /// In other cases ThreadStatus must be accessed only from the current_thread @@ -127,9 +125,29 @@ void CurrentThread::attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thr parent_counters = sibling_thread->performance_counters.getParent(); parent_memory_tracker = sibling_thread->memory_tracker.getParent(); } + logs_queue_ptr = sibling_thread->logs_queue_ptr; } - current_thread->attachQuery(parent_query, parent_counters, parent_memory_tracker, check_detached); + thread->attachQuery(parent_query, parent_counters, parent_memory_tracker, logs_queue_ptr, check_detached); +} + +void CurrentThread::attachSystemLogsQueue(const std::shared_ptr & logs_queue) +{ + getCurrentThreadImpl()->attachSystemLogsQueue(logs_queue); +} + +std::shared_ptr CurrentThread::getSystemLogsQueue() +{ + /// NOTE: this method could be called at early server startup stage + /// NOTE: this method could be called in ThreadStatus destructor, therefore we make use_count() check + + if (!current_thread || current_thread.use_count() <= 0) + return nullptr; + + if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died) + return nullptr; + + return current_thread->getSystemLogsQueue(); } } diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index 9f191be7d34..1d9ed769dc6 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -17,6 +17,7 @@ class QueryStatus; class ThreadStatus; struct Progress; using ThreadStatusPtr = std::shared_ptr; +class SystemLogsQueue; class CurrentThread @@ -29,6 +30,10 @@ public: /// Currently it is used only for debugging static void initializeQuery(); + /// A logs queue used by TCPHandler to pass logs to a client + static void attachSystemLogsQueue(const std::shared_ptr & logs_queue); + static std::shared_ptr getSystemLogsQueue(); + /// You must call one of these methods when create a child thread: /// Bundles the current thread with a query static void attachQuery(QueryStatus * parent_process); diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 88eae23da44..a117d9ed692 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -1,5 +1,4 @@ #include "ThreadStatus.h" -#include #include #include #include @@ -7,6 +6,8 @@ #include #include +#include +#include #include #include @@ -48,18 +49,7 @@ class ThreadStatus::CurrentThreadScope { public: - CurrentThreadScope() - { - try - { - ThreadStatus & thread = *CurrentThread::get(); - LOG_DEBUG(thread.log, "Thread " << thread.thread_number << " started"); - } - catch (...) - { - std::terminate(); - } - } + CurrentThreadScope() = default; ~CurrentThreadScope() { @@ -67,12 +57,12 @@ public: { ThreadStatus & thread = *CurrentThread::get(); + LOG_DEBUG(thread.log, "Thread " << thread.thread_number << " exited"); + if (thread.getCurrentState() != ThreadStatus::ThreadState::DetachedFromQuery) thread.detachQuery(true); else thread.thread_state = ThreadStatus::ThreadState::Died; - - LOG_DEBUG(thread.log, "Thread " << thread.thread_number << " exited"); } catch (...) { @@ -218,7 +208,8 @@ ThreadStatus::ThreadStatus() last_taskstats = std::make_unique(); taskstats_getter = std::make_unique(); - LOG_DEBUG(log, "Thread " << thread_number << " created"); + /// NOTE: It is important not to do any non-trivial actions (like updating ProfileEvents or logging) before ThreadStatus is created + /// Otherwise it could lead to SIGSEGV due to current_thread dereferencing } ThreadStatusPtr ThreadStatus::create() @@ -226,10 +217,7 @@ ThreadStatusPtr ThreadStatus::create() return ThreadStatusPtr(new ThreadStatus); } -ThreadStatus::~ThreadStatus() -{ - LOG_DEBUG(log, "Thread " << thread_number << " destroyed"); -} +ThreadStatus::~ThreadStatus() = default; void ThreadStatus::initializeQuery() { @@ -243,6 +231,7 @@ void ThreadStatus::attachQuery( QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker, + const SystemLogsQueueWeakPtr & logs_queue_ptr_, bool check_detached) { if (thread_state == ThreadState::AttachedToQuery) @@ -261,6 +250,7 @@ void ThreadStatus::attachQuery( performance_counters.setParent(parent_counters); memory_tracker.setParent(parent_memory_tracker); memory_tracker.setDescription("(for thread)"); + logs_queue_ptr = logs_queue_ptr_; } /// Try extract as many information as possible from ProcessList diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 37742b7b6d0..7d920597191 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -22,9 +22,12 @@ class QueryThreadLog; struct TasksStatsCounters; struct RusageCounters; class TaskStatsInfoGetter; +class SystemLogsQueue; +using SystemLogsQueuePtr = std::shared_ptr; +using SystemLogsQueueWeakPtr = std::weak_ptr; + + using ThreadStatusPtr = std::shared_ptr; - - extern thread_local ThreadStatusPtr current_thread; @@ -69,6 +72,18 @@ public: return global_context.load(std::memory_order_relaxed); } + SystemLogsQueuePtr getSystemLogsQueue() const + { + std::lock_guard lock(mutex); + return thread_state == Died ? nullptr : logs_queue_ptr.lock(); + } + + void attachSystemLogsQueue(const SystemLogsQueuePtr & logs_queue) + { + std::lock_guard lock(mutex); + logs_queue_ptr = logs_queue; + } + ~ThreadStatus(); protected: @@ -81,6 +96,7 @@ protected: QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker, + const SystemLogsQueueWeakPtr & logs_queue_ptr_, bool check_detached = true); void detachQuery(bool thread_exits = false); @@ -91,7 +107,7 @@ protected: std::atomic thread_state{ThreadState::DetachedFromQuery}; - std::mutex mutex; + mutable std::mutex mutex; QueryStatus * parent_query = nullptr; /// Is set once @@ -99,6 +115,9 @@ protected: /// Use it only from current thread Context * query_context = nullptr; + /// A logs queue used by TCPHandler to pass logs to a client + SystemLogsQueueWeakPtr logs_queue_ptr; + UInt64 query_start_time_nanoseconds = 0; time_t query_start_time = 0; @@ -111,7 +130,7 @@ protected: friend class CurrentThread; friend struct TasksStatsCounters; - /// Use ptr to not add extra dependencies in header + /// Use ptr not to add extra dependencies in the header std::unique_ptr last_rusage; std::unique_ptr last_taskstats; std::unique_ptr taskstats_getter; diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 7f3a37613dc..16632ccefc3 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -61,6 +61,7 @@ #define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226 #define DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE 54337 #define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372 +#define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54382 /// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change. #define DBMS_TCP_PROTOCOL_VERSION 54226 diff --git a/dbms/src/Core/Protocol.h b/dbms/src/Core/Protocol.h index 458744a3470..5451e1550f6 100644 --- a/dbms/src/Core/Protocol.h +++ b/dbms/src/Core/Protocol.h @@ -78,8 +78,8 @@ namespace Protocol /// See https://www.securecoding.cert.org/confluence/display/cplusplus/INT36-CPP.+Do+not+use+out-of-range+enumeration+values inline const char * toString(UInt64 packet) { - static const char * data[] = { "Hello", "Data", "Exception", "Progress", "Pong", "EndOfStream", "ProfileInfo", "Totals", "Extremes", "TablesStatusResponse" }; - return packet < 10 + static const char * data[] = { "Hello", "Data", "Exception", "Progress", "Pong", "EndOfStream", "ProfileInfo", "Totals", "Extremes", "TablesStatusResponse", "Log" }; + return packet < 11 ? data[packet] : "Unknown packet"; } diff --git a/dbms/src/Core/SystemLogsQueue.cpp b/dbms/src/Core/SystemLogsQueue.cpp new file mode 100644 index 00000000000..aa7e336dc52 --- /dev/null +++ b/dbms/src/Core/SystemLogsQueue.cpp @@ -0,0 +1,55 @@ +#include "SystemLogsQueue.h" +#include +#include +#include +#include + +#include + + +namespace DB +{ + +SystemLogsQueue::SystemLogsQueue() + : ConcurrentBoundedQueue(std::numeric_limits::max()), + max_priority(Poco::Message::Priority::PRIO_INFORMATION) {} + + +Block SystemLogsQueue::getSampleBlock() +{ + return Block { + {std::make_shared(), "event_time"}, + {std::make_shared(), "event_time_microseconds"}, + {std::make_shared(), "thread_number"}, + {std::make_shared(), "priority"}, + {std::make_shared(), "source"}, + {std::make_shared(), "text"}, + }; +} + +MutableColumns SystemLogsQueue::getSampleColumns() +{ + static Block sample_block = getSampleBlock(); + return sample_block.cloneEmptyColumns(); +} + +const char * SystemLogsQueue::getProrityName(int priority) +{ + /// See Poco::Message::Priority + + static const char * PRIORITIES [] = { + "Unknown", + "Fatal", + "Critical", + "Error", + "Warning", + "Notice", + "Information", + "Debug", + "Trace" + }; + + return (priority >= 1 && priority <= 8) ? PRIORITIES[priority] : PRIORITIES[0]; +} + +} diff --git a/dbms/src/Core/SystemLogsQueue.h b/dbms/src/Core/SystemLogsQueue.h new file mode 100644 index 00000000000..f6b8d1b8d69 --- /dev/null +++ b/dbms/src/Core/SystemLogsQueue.h @@ -0,0 +1,29 @@ +#pragma once +#include +#include + + +namespace DB +{ + +class SystemLogsQueue : public ConcurrentBoundedQueue +{ +public: + /// You should not push logs in the queue if their priority greater max_priority + int max_priority; + + SystemLogsQueue(); + + static Block getSampleBlock(); + static MutableColumns getSampleColumns(); + + /// Converts priority from Poco::Message::Priority to a string + static const char * getProrityName(int priority); +}; + +using SystemLogsQueuePtr = std::shared_ptr; + +} + + + diff --git a/dbms/src/DataStreams/BlockIO.h b/dbms/src/DataStreams/BlockIO.h index 685fdef87c2..9f69f834e5f 100644 --- a/dbms/src/DataStreams/BlockIO.h +++ b/dbms/src/DataStreams/BlockIO.h @@ -37,12 +37,17 @@ struct BlockIO exception_callback(); } + /// We provide the correct order of destruction. + void reset() + { + out.reset(); + in.reset(); + process_list_entry.reset(); + } + BlockIO & operator= (const BlockIO & rhs) { - /// We provide the correct order of destruction. - out = nullptr; - in = nullptr; - process_list_entry = nullptr; + reset(); process_list_entry = rhs.process_list_entry; in = rhs.in; diff --git a/dbms/src/DataStreams/SystemLogsRowOutputStream.cpp b/dbms/src/DataStreams/SystemLogsRowOutputStream.cpp new file mode 100644 index 00000000000..4344dc8986a --- /dev/null +++ b/dbms/src/DataStreams/SystemLogsRowOutputStream.cpp @@ -0,0 +1,63 @@ +#include "SystemLogsRowOutputStream.h" +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +Block SystemLogsRowOutputStream::getHeader() const +{ + return SystemLogsQueue::getSampleBlock(); +} + + +void SystemLogsRowOutputStream::write(const Block & block) +{ + for (size_t i = 0; i < block.rows(); ++i) + write(block, i); +} + + +void SystemLogsRowOutputStream::write(const Block & block, size_t row_num) +{ + UInt32 event_time = typeid_cast(*block.getByName("event_time").column).getData()[row_num]; + writeDateTimeText<'.', ':'>(event_time, wb); + + UInt32 microseconds = typeid_cast(*block.getByName("event_time_microseconds").column).getData()[row_num]; + writeChar('.', wb); + writeChar('0' + ((microseconds / 100000) % 10), wb); + writeChar('0' + ((microseconds / 10000) % 10), wb); + writeChar('0' + ((microseconds / 1000) % 10), wb); + writeChar('0' + ((microseconds / 100) % 10), wb); + writeChar('0' + ((microseconds / 10) % 10), wb); + writeChar('0' + ((microseconds / 1) % 10), wb); + + writeCString(" [ ", wb); + UInt32 thread_number = typeid_cast(*block.getByName("thread_number").column).getData()[row_num]; + writeIntText(thread_number, wb); + writeCString(" ] <", wb); + + Int8 priority = typeid_cast(*block.getByName("priority").column).getData()[row_num]; + writeString(SystemLogsQueue::getProrityName(priority), wb); + writeCString("> ", wb); + + writeString(typeid_cast(*block.getByName("source").column).getDataAt(row_num), wb); + writeCString(": ", wb); + writeString(typeid_cast(*block.getByName("text").column).getDataAt(row_num), wb); + + writeChar('\n', wb); +} + +BlockOutputStreamPtr SystemLogsRowOutputStream::create(WriteBuffer & buf_out) +{ + return std::make_shared(buf_out); +} + +} diff --git a/dbms/src/DataStreams/SystemLogsRowOutputStream.h b/dbms/src/DataStreams/SystemLogsRowOutputStream.h new file mode 100644 index 00000000000..d94e209ea49 --- /dev/null +++ b/dbms/src/DataStreams/SystemLogsRowOutputStream.h @@ -0,0 +1,36 @@ +#pragma once +#include +#include + + +namespace DB +{ + +/// Prints internal server logs +/// Input blocks have to have the same structure as SystemLogsQueue::getSampleBlock() +/// NOTE: IRowOutputStream does not suite well for this case +class SystemLogsRowOutputStream : public IBlockOutputStream +{ +public: + + static BlockOutputStreamPtr create(WriteBuffer & buf_out); + + SystemLogsRowOutputStream(WriteBuffer & buf_out) : wb(buf_out) {} + + Block getHeader() const override; + + void write(const Block & block, size_t row_num); + + void write(const Block & block) override; + + void flush() override + { + wb.next(); + } + +private: + + WriteBuffer & wb; +}; + +} diff --git a/dbms/src/Interpreters/ClickHouseLogChannel.cpp b/dbms/src/Interpreters/ClickHouseLogChannel.cpp new file mode 100644 index 00000000000..0fc15550f1d --- /dev/null +++ b/dbms/src/Interpreters/ClickHouseLogChannel.cpp @@ -0,0 +1,45 @@ +#include "ClickHouseLogChannel.h" + +#include +#include + +#include +#include + +#include +#include + + +namespace DB +{ + + +void ClickHouseLogChannel::log(const Poco::Message & msg) +{ + if (auto logs_queue = CurrentThread::getSystemLogsQueue()) + { + /// Too noisy message + if (msg.getPriority() > logs_queue->max_priority) + return; + + MutableColumns columns = SystemLogsQueue::getSampleColumns(); + + /// TODO: it would be better if the time was exactly the same as one in OwnPatternFormatter + ::timeval tv; + if (0 != gettimeofday(&tv, nullptr)) + DB::throwFromErrno("Cannot gettimeofday"); + + size_t i = 0; + columns[i++]->insert(static_cast(tv.tv_sec)); + columns[i++]->insert(static_cast(tv.tv_usec)); + columns[i++]->insert(static_cast(Poco::ThreadNumber::get())); + columns[i++]->insert(static_cast(msg.getPriority())); + columns[i++]->insert(msg.getSource()); + columns[i++]->insert(msg.getText()); + + logs_queue->emplace(std::move(columns)); + } +} + + +} diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index c4dcffee3d9..9e149c59c3c 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -258,7 +258,8 @@ struct Settings M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \ M(SettingBool, log_profile_events, true, "Log query settings into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query performance statistics into the query_log.") \ - M(SettingBool, log_query_threads, true, "Log query threads into system.query_log table.") + M(SettingBool, log_query_threads, true, "Log query threads into system.query_log table.") \ + M(SettingString, server_logs_level, "warning", "Send internal server logs to client.") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Server/CMakeLists.txt b/dbms/src/Server/CMakeLists.txt index 747cab4c94a..53a9cc4c50f 100644 --- a/dbms/src/Server/CMakeLists.txt +++ b/dbms/src/Server/CMakeLists.txt @@ -24,8 +24,7 @@ add_library (clickhouse-server-lib RootRequestHandler.cpp Server.cpp StatusFile.cpp - TCPHandler.cpp - ) + TCPHandler.cpp) target_link_libraries (clickhouse-server-lib clickhouse_common_io daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions) target_include_directories (clickhouse-server-lib PUBLIC ${ClickHouse_SOURCE_DIR}/libs/libdaemon/include) @@ -101,7 +100,7 @@ if (CLICKHOUSE_SPLIT_BINARY) add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_ALL_TARGETS}) add_custom_target (clickhouse ALL DEPENDS clickhouse-bundle) else () - add_executable (clickhouse main.cpp) + add_executable (clickhouse main.cpp ../Interpreters/ClickHouseLogChannel.cpp) target_link_libraries (clickhouse clickhouse_common_io) target_include_directories (clickhouse BEFORE PRIVATE ${COMMON_INCLUDE_DIR}) target_include_directories (clickhouse PRIVATE ${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Server/Client.cpp b/dbms/src/Server/Client.cpp index e2f8b28957e..f55302fbccb 100644 --- a/dbms/src/Server/Client.cpp +++ b/dbms/src/Server/Client.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include #include @@ -131,10 +132,12 @@ private: /// Console output. WriteBufferFromFileDescriptor std_out {STDOUT_FILENO}; + WriteBufferFromFileDescriptor std_err {STDERR_FILENO}; std::unique_ptr pager_cmd; /// The user can specify to redirect query output to a file. std::optional out_file_buf; BlockOutputStreamPtr block_out_stream; + BlockOutputStreamPtr logs_out_stream; String home_path; @@ -963,18 +966,23 @@ private: void resetOutput() { block_out_stream = nullptr; + logs_out_stream = nullptr; + if (pager_cmd) { pager_cmd->in.close(); pager_cmd->wait(); } pager_cmd = nullptr; + if (out_file_buf) { out_file_buf->next(); out_file_buf.reset(); } + std_out.next(); + std_err.next(); } @@ -1048,6 +1056,10 @@ private: last_exception = std::move(packet.exception); return false; + case Protocol::Server::Log: + onLogData(packet.block); + return true; + case Protocol::Server::EndOfStream: onEndOfStream(); return false; @@ -1132,6 +1144,16 @@ private: } + void initLogsOutputStream() + { + if (!logs_out_stream) + { + logs_out_stream = SystemLogsRowOutputStream::create(std_err); + logs_out_stream->writePrefix(); + } + } + + void onData(Block & block) { if (written_progress_chars) @@ -1155,6 +1177,14 @@ private: } + void onLogData(Block & block) + { + initLogsOutputStream(); + logs_out_stream->write(block); + logs_out_stream->flush(); + } + + void onTotals(Block & block) { initBlockOutputStream(block); @@ -1307,6 +1337,9 @@ private: if (block_out_stream) block_out_stream->writeSuffix(); + if (logs_out_stream) + logs_out_stream->writeSuffix(); + resetOutput(); if (is_interactive && !written_first_block) @@ -1385,7 +1418,9 @@ public: ioctl(0, TIOCGWINSZ, &terminal_size); - unsigned line_length = boost::program_options::options_description::m_default_line_length; + namespace po = boost::program_options; + + unsigned line_length = po::options_description::m_default_line_length; unsigned min_description_length = line_length / 2; if (!stdin_is_not_tty) { @@ -1393,54 +1428,55 @@ public: min_description_length = std::min(min_description_length, line_length - 2); } -#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value (), DESCRIPTION) +#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, po::value (), DESCRIPTION) /// Main commandline options related to client functionality and all parameters from Settings. - boost::program_options::options_description main_description("Main options", line_length, min_description_length); + po::options_description main_description("Main options", line_length, min_description_length); main_description.add_options() ("help", "produce help message") - ("config-file,c", boost::program_options::value(), "config-file path") - ("host,h", boost::program_options::value()->default_value("localhost"), "server host") - ("port", boost::program_options::value()->default_value(9000), "server port") + ("config-file,c", po::value(), "config-file path") + ("host,h", po::value()->default_value("localhost"), "server host") + ("port", po::value()->default_value(9000), "server port") ("secure,s", "secure") - ("user,u", boost::program_options::value()->default_value("default"), "user") - ("password", boost::program_options::value(), "password") + ("user,u", po::value()->default_value("default"), "user") + ("password", po::value(), "password") ("ask-password", "ask-password") - ("query_id", boost::program_options::value(), "query_id") - ("query,q", boost::program_options::value(), "query") - ("database,d", boost::program_options::value(), "database") - ("pager", boost::program_options::value(), "pager") + ("query_id", po::value(), "query_id") + ("query,q", po::value(), "query") + ("database,d", po::value(), "database") + ("pager", po::value(), "pager") ("multiline,m", "multiline") ("multiquery,n", "multiquery") ("ignore-error", "Do not stop processing in multiquery mode") - ("format,f", boost::program_options::value(), "default output format") + ("format,f", po::value(), "default output format") ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") ("stacktrace", "print stack traces of exceptions") ("progress", "print progress even in non-interactive mode") ("version,V", "print version information and exit") ("echo", "in batch mode, print query before execution") - ("max_client_network_bandwidth", boost::program_options::value(), "the maximum speed of data exchange over the network for the client in bytes per second.") - ("compression", boost::program_options::value(), "enable or disable compression") + ("max_client_network_bandwidth", po::value(), "the maximum speed of data exchange over the network for the client in bytes per second.") + ("compression", po::value(), "enable or disable compression") + ("log-level", po::value(), "log level") APPLY_FOR_SETTINGS(DECLARE_SETTING) ; #undef DECLARE_SETTING /// Commandline options related to external tables. - boost::program_options::options_description external_description("External tables options"); + po::options_description external_description("External tables options"); external_description.add_options() - ("file", boost::program_options::value(), "data file or - for stdin") - ("name", boost::program_options::value()->default_value("_data"), "name of the table") - ("format", boost::program_options::value()->default_value("TabSeparated"), "data format") - ("structure", boost::program_options::value(), "structure") - ("types", boost::program_options::value(), "types") + ("file", po::value(), "data file or - for stdin") + ("name", po::value()->default_value("_data"), "name of the table") + ("format", po::value()->default_value("TabSeparated"), "data format") + ("structure", po::value(), "structure") + ("types", po::value(), "types") ; /// Parse main commandline options. - boost::program_options::parsed_options parsed = boost::program_options::command_line_parser( + po::parsed_options parsed = po::command_line_parser( common_arguments.size(), common_arguments.data()).options(main_description).run(); - boost::program_options::variables_map options; - boost::program_options::store(parsed, options); + po::variables_map options; + po::store(parsed, options); if (options.count("version") || options.count("V")) { @@ -1457,14 +1493,17 @@ public: exit(0); } + if (options.count("log-level")) + Poco::Logger::root().setLevel(options["log-level"].as()); + size_t number_of_external_tables_with_stdin_source = 0; for (size_t i = 0; i < external_tables_arguments.size(); ++i) { /// Parse commandline options related to external tables. - boost::program_options::parsed_options parsed = boost::program_options::command_line_parser( + po::parsed_options parsed = po::command_line_parser( external_tables_arguments[i].size(), external_tables_arguments[i].data()).options(external_description).run(); - boost::program_options::variables_map external_options; - boost::program_options::store(parsed, external_options); + po::variables_map external_options; + po::store(parsed, external_options); try { @@ -1555,6 +1594,11 @@ int mainEntryClickHouseClient(int argc, char ** argv) std::cerr << "Bad arguments: " << e.what() << std::endl; return 1; } + catch (...) + { + std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; + return 1; + } return client.run(); } diff --git a/dbms/src/Server/ClusterCopier.cpp b/dbms/src/Server/ClusterCopier.cpp index 2cf6088b494..22f483d45d3 100644 --- a/dbms/src/Server/ClusterCopier.cpp +++ b/dbms/src/Server/ClusterCopier.cpp @@ -65,7 +65,6 @@ #include #include #include -#include namespace DB diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 1e1a911fb19..df320adc79c 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -29,6 +29,8 @@ #include #include "TCPHandler.h" +#include +#include #include #include @@ -152,6 +154,15 @@ void TCPHandler::runImpl() CurrentThread::initializeQuery(); + /// Should we send internal logs to client? + if (client_revision >= DBMS_MIN_REVISION_WITH_SERVER_LOGS + && !query_context.getSettingsRef().server_logs_level.value.empty()) + { + state.logs_queue = std::make_shared(); + state.logs_queue->max_priority = Poco::Logger::parseLevel(query_context.getSettingsRef().server_logs_level.value); + CurrentThread::attachSystemLogsQueue(state.logs_queue); + } + query_context.setExternalTablesInitializer([&global_settings, this] (Context & context) { if (&context != &query_context) throw Exception("Unexpected context in external tables initializer", ErrorCodes::LOGICAL_ERROR); @@ -180,8 +191,11 @@ void TCPHandler::runImpl() else processOrdinaryQuery(); - sendEndOfStream(); + /// Reset BlockIO in advance to log destruction actions + state.io.reset(); + sendLogs(); + sendEndOfStream(); state.reset(); } catch (const Exception & e) @@ -226,7 +240,10 @@ void TCPHandler::runImpl() try { if (exception) + { + sendLogs(); sendException(*exception); + } } catch (...) { @@ -269,12 +286,14 @@ void TCPHandler::readData(const Settings & global_settings) constexpr size_t min_poll_interval = 5000; // 5 ms size_t poll_interval = std::max(min_poll_interval, std::min(default_poll_interval, current_poll_interval)); - while (1) + while (true) { Stopwatch watch(CLOCK_MONOTONIC_COARSE); + sendLogs(); + /// We are waiting for a packet from the client. Thus, every `POLL_INTERVAL` seconds check whether we need to shut down. - while (1) + while (true) { if (static_cast(*in).poll(poll_interval)) break; @@ -363,6 +382,8 @@ void TCPHandler::processOrdinaryQuery() sendProgress(); } + sendLogs(); + if (async_in.poll(query_context.getSettingsRef().interactive_delay / 1000)) { /// There is the following result block. @@ -385,6 +406,7 @@ void TCPHandler::processOrdinaryQuery() sendExtremes(); sendProfileInfo(); sendProgress(); + sendLogs(); } sendData(block); @@ -708,11 +730,7 @@ void TCPHandler::initBlockOutput(const Block & block) { if (!state.block_out) { - if (state.compression == Protocol::Compression::Enable) - state.maybe_compressed_out = std::make_shared( - *out, CompressionSettings(query_context.getSettingsRef())); - else - state.maybe_compressed_out = out; + initOutputBuffers(); state.block_out = std::make_shared( *state.maybe_compressed_out, @@ -721,6 +739,32 @@ void TCPHandler::initBlockOutput(const Block & block) } } +void TCPHandler::initLogsBlockOutput(const Block & block) +{ + if (!state.logs_block_out) + { + initOutputBuffers(); + + state.logs_block_out = std::make_shared( + *state.maybe_compressed_out, + client_revision, + block.cloneEmpty()); + } +} + + +void TCPHandler::initOutputBuffers() +{ + if (!state.maybe_compressed_out) + { + if (state.compression == Protocol::Compression::Enable) + state.maybe_compressed_out = std::make_shared( + *out, CompressionSettings(query_context.getSettingsRef())); + else + state.maybe_compressed_out = out; + } +} + bool TCPHandler::isQueryCancelled() { @@ -761,6 +805,7 @@ void TCPHandler::sendData(const Block & block) initBlockOutput(block); writeVarUInt(Protocol::Server::Data, *out); + /// Send external table name (empty name is the main table) writeStringBinary("", *out); state.block_out->write(block); @@ -769,6 +814,20 @@ void TCPHandler::sendData(const Block & block) } +void TCPHandler::sendLogData(const Block & block) +{ + initLogsBlockOutput(block); + + writeVarUInt(Protocol::Server::Log, *out); + /// Send log tag (empty tag is the default tag) + writeStringBinary("", *out); + + state.logs_block_out->write(block); + state.maybe_compressed_out->next(); + out->next(); +} + + void TCPHandler::sendException(const Exception & e) { writeVarUInt(Protocol::Server::Exception, *out); @@ -800,6 +859,41 @@ void TCPHandler::sendProgress() } +void TCPHandler::sendLogs() +{ + if (!state.logs_queue) + return; + + MutableColumns logs_columns; + MutableColumns curr_logs_columns; + size_t rows = 0; + + for (; state.logs_queue->tryPop(curr_logs_columns); ++rows) + { + if (rows == 0) + { + logs_columns = std::move(curr_logs_columns); + } + else + { + for (size_t j = 0; j < logs_columns.size(); ++j) + logs_columns[j]->insertRangeFrom(*curr_logs_columns[j], 0, curr_logs_columns[j]->size()); + } + } + + if (rows > 0) + { + Block block = SystemLogsQueue::getSampleBlock(); + block.setColumns(std::move(logs_columns)); + block.checkNumberOfRows(); + + std::cerr << "sendLogs: " << block.rows() << " " << block.columns() << "\n"; + + sendLogData(block); + } +} + + void TCPHandler::run() { try diff --git a/dbms/src/Server/TCPHandler.h b/dbms/src/Server/TCPHandler.h index e01987d3bbd..1c184876714 100644 --- a/dbms/src/Server/TCPHandler.h +++ b/dbms/src/Server/TCPHandler.h @@ -8,12 +8,17 @@ #include #include #include +#include #include #include #include #include #include "IServer.h" +#include "../Common/ZooKeeper/Increment.h" +#include "../Core/Block.h" +#include "../Common/Exception.h" + namespace CurrentMetrics { @@ -63,6 +68,9 @@ struct QueryState /// Timeouts setter for current query std::unique_ptr timeout_setter; + /// A queue with internal logs that will be passed to client + SystemLogsQueuePtr logs_queue; + BlockOutputStreamPtr logs_block_out; void reset() { @@ -139,8 +147,10 @@ private: void sendHello(); void sendData(const Block & block); /// Write a block to the network. + void sendLogData(const Block & block); void sendException(const Exception & e); void sendProgress(); + void sendLogs(); void sendEndOfStream(); void sendProfileInfo(); void sendTotals(); @@ -149,6 +159,8 @@ private: /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. void initBlockInput(); void initBlockOutput(const Block & block); + void initLogsBlockOutput(const Block & block); + void initOutputBuffers(); bool isQueryCancelled(); diff --git a/libs/libdaemon/include/daemon/OwnPatternFormatter.h b/libs/libdaemon/include/daemon/OwnPatternFormatter.h index 63e14a34d84..11c195f2171 100644 --- a/libs/libdaemon/include/daemon/OwnPatternFormatter.h +++ b/libs/libdaemon/include/daemon/OwnPatternFormatter.h @@ -23,6 +23,7 @@ class BaseDaemon; class OwnPatternFormatter : public Poco::PatternFormatter { public: + /// ADD_LAYER_TAG is needed only for Yandex.Metrika, that share part of ClickHouse code. enum Options { diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 0d2b53fd9bc..dd85e7b756d 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -62,6 +63,7 @@ #include #include #include +#include using Poco::Logger; @@ -778,6 +780,11 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) split->addChannel(log); } + { + Poco::AutoPtr internal_logger = new DB::ClickHouseLogChannel; + split->addChannel(internal_logger); + } + split->open(); logger().close(); logger().setChannel(split); @@ -1047,6 +1054,7 @@ void BaseDaemon::initialize(Application & self) static KillingErrorHandler killing_error_handler; Poco::ErrorHandler::set(&killing_error_handler); + DB::CurrentThread::get(); logRevision(); signal_listener.reset(new SignalListener(*this)); diff --git a/libs/libdaemon/src/OwnPatternFormatter.cpp b/libs/libdaemon/src/OwnPatternFormatter.cpp index fbb3d0c2061..cb0c745a375 100644 --- a/libs/libdaemon/src/OwnPatternFormatter.cpp +++ b/libs/libdaemon/src/OwnPatternFormatter.cpp @@ -1,8 +1,11 @@ #include -#include #include #include +#include +#include + +#include #include #include #include @@ -26,7 +29,7 @@ void OwnPatternFormatter::format(const Poco::Message & msg, std::string & text) } /// Output time with microsecond resolution. - timeval tv; + ::timeval tv; if (0 != gettimeofday(&tv, nullptr)) DB::throwFromErrno("Cannot gettimeofday"); From 99b68d3cf367c7a8fbad70c70f1a3290afc70052 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Thu, 7 Jun 2018 21:46:53 +0300 Subject: [PATCH 077/315] Add more options for loggig. [#CLICKHOUSE-2910] --- dbms/src/Interpreters/Settings.h | 2 +- dbms/src/Server/Client.cpp | 43 ++++++++++++++++++++++++++++---- 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 9e149c59c3c..f2057b628c1 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -259,7 +259,7 @@ struct Settings M(SettingBool, log_profile_events, true, "Log query settings into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query performance statistics into the query_log.") \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_log table.") \ - M(SettingString, server_logs_level, "warning", "Send internal server logs to client.") + M(SettingString, server_logs_level, "", "Send internal server logs to client.") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Server/Client.cpp b/dbms/src/Server/Client.cpp index f55302fbccb..8e0a140eb4d 100644 --- a/dbms/src/Server/Client.cpp +++ b/dbms/src/Server/Client.cpp @@ -132,11 +132,14 @@ private: /// Console output. WriteBufferFromFileDescriptor std_out {STDOUT_FILENO}; - WriteBufferFromFileDescriptor std_err {STDERR_FILENO}; std::unique_ptr pager_cmd; /// The user can specify to redirect query output to a file. std::optional out_file_buf; BlockOutputStreamPtr block_out_stream; + + /// The user could specify special file for server logs (stderr by default) + std::unique_ptr out_logs_buf; + String server_logs_file; BlockOutputStreamPtr logs_out_stream; String home_path; @@ -965,8 +968,8 @@ private: /// Flush all buffers. void resetOutput() { - block_out_stream = nullptr; - logs_out_stream = nullptr; + block_out_stream.reset(); + logs_out_stream.reset(); if (pager_cmd) { @@ -981,8 +984,13 @@ private: out_file_buf.reset(); } + if (out_logs_buf) + { + out_logs_buf->next(); + out_logs_buf.reset(); + } + std_out.next(); - std_err.next(); } @@ -1148,7 +1156,29 @@ private: { if (!logs_out_stream) { - logs_out_stream = SystemLogsRowOutputStream::create(std_err); + WriteBuffer * wb = out_logs_buf.get(); + + if (!out_logs_buf) + { + if (server_logs_file.empty()) + { + /// Use stderr by default + out_logs_buf = std::make_unique(STDERR_FILENO); + wb = out_logs_buf.get(); + } + else if (server_logs_file == "-") + { + /// Use stdout if --server_logs_file=- specified + wb = &std_out; + } + else + { + out_logs_buf = std::make_unique(server_logs_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); + wb = out_logs_buf.get(); + } + } + + logs_out_stream = SystemLogsRowOutputStream::create(*wb); logs_out_stream->writePrefix(); } } @@ -1458,6 +1488,7 @@ public: ("max_client_network_bandwidth", po::value(), "the maximum speed of data exchange over the network for the client in bytes per second.") ("compression", po::value(), "enable or disable compression") ("log-level", po::value(), "log level") + ("server_logs_file", po::value(), "put server logs into specified file") APPLY_FOR_SETTINGS(DECLARE_SETTING) ; #undef DECLARE_SETTING @@ -1575,6 +1606,8 @@ public: max_client_network_bandwidth = options["max_client_network_bandwidth"].as(); if (options.count("compression")) config().setBool("compression", options["compression"].as()); + if (options.count("server_logs_file")) + server_logs_file = options["server_logs_file"].as(); } }; From ed7cd86f09a5309f584440d89ecdca69588db0da Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Fri, 8 Jun 2018 22:50:15 +0300 Subject: [PATCH 078/315] Fixed several bug. Trun on --server_logs_level=warning in tests. [#CLICKHOUSE-2910] Fixed several test to be compatible with --server_logs_level=warning. --- dbms/src/Client/Connection.cpp | 34 ++++++++- dbms/src/Client/Connection.h | 5 ++ dbms/src/Client/MultiplexedConnections.cpp | 1 + .../DataStreams/RemoteBlockInputStream.cpp | 3 + .../DataStreams/RemoteBlockOutputStream.cpp | 61 +++++++++------ dbms/src/Interpreters/Aggregator.cpp | 8 +- dbms/src/Interpreters/Settings.h | 2 +- dbms/src/Server/Client.cpp | 74 +++++++++++++++---- dbms/src/Server/TCPHandler.cpp | 6 +- dbms/tests/clickhouse-test | 4 +- .../queries/0_stateless/00155_long_merges.sh | 2 +- .../00158_buffer_and_nonexistent_table.sql | 1 + .../00183_skip_unavailable_shards.sql | 1 + .../0_stateless/00336_shard_stack_trace.sh | 6 +- .../00443_optimize_final_vertical_merge.sh | 2 +- .../0_stateless/00474_readonly_settings.sh | 4 +- .../00492_drop_temporary_table.sql | 2 + .../0_stateless/00550_join_insert_select.sh | 2 +- .../00553_buff_exists_materlized_column.sql | 2 +- .../0_stateless/00568_compile_catch_throw.sh | 2 +- ...column_exception_when_drop_depen_column.sh | 2 +- .../0_stateless/00595_insert_into_view.sh | 2 +- .../00596_limit_on_expanded_ast.sh | 2 +- .../queries/0_stateless/00602_throw_if.sh | 4 +- .../00626_replace_partition_from_table.sql | 2 +- ...replace_partition_from_table_zookeeper.sql | 2 +- dbms/tests/queries/shell_config.sh | 2 +- 27 files changed, 167 insertions(+), 71 deletions(-) diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 3c8912e4d3d..cbd588a25d1 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -111,6 +111,7 @@ void Connection::disconnect() //LOG_TRACE(log_wrapper.get(), "Disconnecting"); in = nullptr; + last_input_packet_type.reset(); out = nullptr; // can write to socket if (socket) socket->close(); @@ -484,7 +485,26 @@ bool Connection::poll(size_t timeout_microseconds) bool Connection::hasReadBufferPendingData() const { - return static_cast(*in).hasPendingData(); + return last_input_packet_type.has_value() || static_cast(*in).hasPendingData(); +} + + +std::optional Connection::checkPacket(size_t timeout_microseconds) +{ + if (last_input_packet_type.has_value()) + return last_input_packet_type; + + if (hasReadBufferPendingData() || poll(timeout_microseconds)) + { + // LOG_TRACE(log_wrapper.get(), "Receiving packet type"); + UInt64 packet_type; + readVarUInt(packet_type, *in); + + last_input_packet_type.emplace(packet_type); + return last_input_packet_type; + } + + return {}; } @@ -493,7 +513,17 @@ Connection::Packet Connection::receivePacket() try { Packet res; - readVarUInt(res.type, *in); + + if (last_input_packet_type) + { + res.type = *last_input_packet_type; + last_input_packet_type.reset(); + } + else + { + LOG_TRACE(log_wrapper.get(), "Receiving packet type"); + readVarUInt(res.type, *in); + } // LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type)); diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index 6a8d0f71781..a7cadecee8d 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -23,6 +23,7 @@ #include #include +#include namespace DB @@ -140,6 +141,9 @@ public: /// Check, if has data in read buffer. bool hasReadBufferPendingData() const; + /// Checks if there is input data in connection and reads packet ID. + std::optional checkPacket(size_t timeout_microseconds = 0); + /// Receive packet from server. Packet receivePacket(); @@ -194,6 +198,7 @@ private: std::unique_ptr socket; std::shared_ptr in; std::shared_ptr out; + std::optional last_input_packet_type; String query_id; Protocol::Compression compression; /// Enable data compression for communication. diff --git a/dbms/src/Client/MultiplexedConnections.cpp b/dbms/src/Client/MultiplexedConnections.cpp index 8fe27ecf7fa..6c534ff53f4 100644 --- a/dbms/src/Client/MultiplexedConnections.cpp +++ b/dbms/src/Client/MultiplexedConnections.cpp @@ -247,6 +247,7 @@ Connection::Packet MultiplexedConnections::receivePacketUnlocked() case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: case Protocol::Server::Extremes: + case Protocol::Server::Log: break; case Protocol::Server::EndOfStream: diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 7b851ec8469..7baff3de4b4 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -232,6 +232,9 @@ Block RemoteBlockInputStream::readImpl() extremes = packet.block; break; + case Protocol::Server::Log: + break; + default: got_unknown_packet_from_replica = true; throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); diff --git a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp index 51fb62ef5a1..9bc0fa3b101 100644 --- a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp @@ -24,23 +24,31 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const */ connection.sendQuery(query, "", QueryProcessingStage::Complete, settings, nullptr); - Connection::Packet packet = connection.receivePacket(); - - if (Protocol::Server::Data == packet.type) + while (true) { - header = packet.block; + Connection::Packet packet = connection.receivePacket(); - if (!header) - throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR); + if (Protocol::Server::Data == packet.type) + { + header = packet.block; + + if (!header) + throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR); + break; + } + else if (Protocol::Server::Exception == packet.type) + { + packet.exception->rethrow(); + break; + } + else if (Protocol::Server::Log == packet.type) + { + /// Do nothing + } + else + throw NetException("Unexpected packet from server (expected Data or Exception, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } - else if (Protocol::Server::Exception == packet.type) - { - packet.exception->rethrow(); - return; - } - else - throw NetException("Unexpected packet from server (expected Data or Exception, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } @@ -83,18 +91,23 @@ void RemoteBlockOutputStream::writeSuffix() /// Empty block means end of data. connection.sendData(Block()); - /// Receive EndOfStream packet. - Connection::Packet packet = connection.receivePacket(); - - if (Protocol::Server::EndOfStream == packet.type) + /// Wait for EndOfStream or Exception packet, skip Log packets. + while (true) { - /// Do nothing. + Connection::Packet packet = connection.receivePacket(); + + if (Protocol::Server::EndOfStream == packet.type) + break; + else if (Protocol::Server::Exception == packet.type) + packet.exception->rethrow(); + else if (Protocol::Server::Log == packet.type) + { + // Do nothing + } + else + throw NetException("Unexpected packet from server (expected EndOfStream or Exception, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } - else if (Protocol::Server::Exception == packet.type) - packet.exception->rethrow(); - else - throw NetException("Unexpected packet from server (expected EndOfStream or Exception, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); finished = true; } diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 65bb2b3a380..0886fc17455 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1726,12 +1726,12 @@ private: void thread(Int32 bucket_num, ThreadStatusPtr main_thread) { - setThreadName("MergingAggregtd"); - CurrentThread::attachQueryFromSiblingThread(main_thread); - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - try { + setThreadName("MergingAggregtd"); + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; + /// TODO: add no_more_keys support maybe auto & merged_data = *data[0]; diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index f2057b628c1..1fc010c80ad 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -259,7 +259,7 @@ struct Settings M(SettingBool, log_profile_events, true, "Log query settings into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query performance statistics into the query_log.") \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_log table.") \ - M(SettingString, server_logs_level, "", "Send internal server logs to client.") + M(SettingString, server_logs_level, "none", "Send internal server logs to client.") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Server/Client.cpp b/dbms/src/Server/Client.cpp index 8e0a140eb4d..b013d195209 100644 --- a/dbms/src/Server/Client.cpp +++ b/dbms/src/Server/Client.cpp @@ -875,7 +875,7 @@ private: /// If structure was received (thus, server has not thrown an exception), /// send our data with that structure. sendData(sample); - receivePacket(); + receiveEndOfQuery(); } } @@ -959,6 +959,11 @@ private: if (!block) break; + + /// Check if server send Log packet + auto packet_type = connection->checkPacket(); + if (packet_type && *packet_type == Protocol::Server::Log) + connection->receivePacket(); } async_block_input->readSuffix(); @@ -1022,7 +1027,7 @@ private: continue; /// If there is no new data, continue checking whether the query was cancelled after a timeout. } - if (!receivePacket()) + if (!receiveAndProcessPacket()) break; } @@ -1033,7 +1038,7 @@ private: /// Receive a part of the result, or progress info or an exception and process it. /// Returns true if one should continue receiving packets. - bool receivePacket() + bool receiveAndProcessPacket() { Connection::Packet packet = connection->receivePacket(); @@ -1081,22 +1086,59 @@ private: /// Receive the block that serves as an example of the structure of table where data will be inserted. bool receiveSampleBlock(Block & out) { - Connection::Packet packet = connection->receivePacket(); - - switch (packet.type) + while (true) { - case Protocol::Server::Data: - out = packet.block; - return true; + Connection::Packet packet = connection->receivePacket(); - case Protocol::Server::Exception: - onException(*packet.exception); - last_exception = std::move(packet.exception); - return false; + switch (packet.type) + { + case Protocol::Server::Data: + out = packet.block; + return true; - default: - throw NetException("Unexpected packet from server (expected Data, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + case Protocol::Server::Exception: + onException(*packet.exception); + last_exception = std::move(packet.exception); + return false; + + case Protocol::Server::Log: + onLogData(packet.block); + break; + + default: + throw NetException("Unexpected packet from server (expected Data, Exception or Log, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + } + } + } + + + /// Process Log packets, exit when recieve Exception or EndOfStream + bool receiveEndOfQuery() + { + while (true) + { + Connection::Packet packet = connection->receivePacket(); + + switch (packet.type) + { + case Protocol::Server::EndOfStream: + onEndOfStream(); + return true; + + case Protocol::Server::Exception: + onException(*packet.exception); + last_exception = std::move(packet.exception); + return false; + + case Protocol::Server::Log: + onLogData(packet.block); + break; + + default: + throw NetException("Unexpected packet from server (expected Exception, EndOfStream or Log, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + } } } diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index df320adc79c..b37e38c696a 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -156,7 +156,7 @@ void TCPHandler::runImpl() /// Should we send internal logs to client? if (client_revision >= DBMS_MIN_REVISION_WITH_SERVER_LOGS - && !query_context.getSettingsRef().server_logs_level.value.empty()) + && query_context.getSettingsRef().server_logs_level.value != "none") { state.logs_queue = std::make_shared(); state.logs_queue->max_priority = Poco::Logger::parseLevel(query_context.getSettingsRef().server_logs_level.value); @@ -885,10 +885,6 @@ void TCPHandler::sendLogs() { Block block = SystemLogsQueue::getSampleBlock(); block.setColumns(std::move(logs_columns)); - block.checkNumberOfRows(); - - std::cerr << "sendLogs: " << block.rows() << " " << block.columns() << "\n"; - sendLogData(block); } } diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 37e1b53c117..3ab93acda99 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -73,7 +73,7 @@ def main(args): base_dir = os.path.abspath(args.queries) tmp_dir = os.path.abspath(args.tmp) - # Keep same default values as in queries/0_stateless/00000_sh_lib.sh + # Keep same default values as in queries/shell_config.sh os.environ.setdefault("CLICKHOUSE_BINARY", args.binary) os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) @@ -364,5 +364,7 @@ if __name__ == '__main__': args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST") if os.getenv("CLICKHOUSE_PORT_TCP"): args.client += ' --port=' + os.getenv("CLICKHOUSE_PORT_TCP") + # Will print all warnings and errors + args.client += ' --server_logs_level=warning' main(args) diff --git a/dbms/tests/queries/0_stateless/00155_long_merges.sh b/dbms/tests/queries/0_stateless/00155_long_merges.sh index d711a18a4b9..a5a636a678a 100755 --- a/dbms/tests/queries/0_stateless/00155_long_merges.sh +++ b/dbms/tests/queries/0_stateless/00155_long_merges.sh @@ -49,7 +49,7 @@ function test { $CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $SUM FROM test.summing" echo $CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sum(s) = $SUM FROM test.collapsing" - $CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test.collapsing" + $CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test.collapsing" --server_logs_file='/dev/null'; $CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $MAX FROM test.collapsing" echo $CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sumMerge(s) = $SUM FROM test.aggregating" diff --git a/dbms/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql b/dbms/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql index 4d3265d7903..e4be7b464d2 100644 --- a/dbms/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql +++ b/dbms/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql @@ -1,6 +1,7 @@ CREATE DATABASE IF NOT EXISTS test2; DROP TABLE IF EXISTS test2.mt_buffer; CREATE TABLE test2.mt_buffer (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2, mt, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000); +SET server_logs_level='none'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded." INSERT INTO test2.mt_buffer (x) SELECT number AS x FROM system.numbers LIMIT 100000; INSERT INTO test2.mt_buffer (x) SELECT number AS x FROM system.numbers LIMIT 1000000; DROP DATABASE test2; diff --git a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql index 36768985c0f..a2f8537c0a4 100644 --- a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql +++ b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql @@ -1 +1,2 @@ +SET server_logs_level='none'; SELECT count() FROM remote('{127,1}.0.0.{2,3}', system.one) SETTINGS skip_unavailable_shards = 1; diff --git a/dbms/tests/queries/0_stateless/00336_shard_stack_trace.sh b/dbms/tests/queries/0_stateless/00336_shard_stack_trace.sh index 941cd5bbe15..f2a2556d328 100755 --- a/dbms/tests/queries/0_stateless/00336_shard_stack_trace.sh +++ b/dbms/tests/queries/0_stateless/00336_shard_stack_trace.sh @@ -8,6 +8,6 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?stacktrace=0" -d 'SELECT a' | wc -l [[ $(${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?stacktrace=1" -d 'SELECT a' | wc -l) -gt 3 ]] && echo 'Ok' || echo 'Fail' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT intDiv(number, 0) FROM remote('127.0.0.{2,3}', system.numbers)" | wc -l -$CLICKHOUSE_CLIENT --query="SELECT a" 2>&1 | wc -l -[[ $($CLICKHOUSE_CLIENT --query="SELECT a" --stacktrace 2>&1 | wc -l) -gt 3 ]] && echo 'Ok' || echo 'Fail' -$CLICKHOUSE_CLIENT --query="SELECT intDiv(number, 0) FROM remote('127.0.0.{2,3}', system.numbers)" 2>&1 | wc -l +$CLICKHOUSE_CLIENT --query="SELECT a" --server_logs_file=/dev/null 2>&1 | wc -l +[[ $($CLICKHOUSE_CLIENT --query="SELECT a" --server_logs_file=/dev/null --stacktrace 2>&1 | wc -l) -gt 3 ]] && echo 'Ok' || echo 'Fail' +$CLICKHOUSE_CLIENT --query="SELECT intDiv(number, 0) FROM remote('127.0.0.{2,3}', system.numbers)" --server_logs_file=/dev/null 2>&1 | wc -l diff --git a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh index 3b171439b5f..43392107b31 100755 --- a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh +++ b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh @@ -75,7 +75,7 @@ while [[ `get_num_parts` -ne 1 ]] ; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $na $CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN n.a Array(String)" $CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN da Array(String) DEFAULT ['def']" -$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001 FINAL" +$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001 FINAL" --server_logs_file=/dev/null $CLICKHOUSE_CLIENT -q "ALTER TABLE $name MODIFY COLUMN n.a Array(String) DEFAULT ['zzz']" $CLICKHOUSE_CLIENT -q "ALTER TABLE $name MODIFY COLUMN da Array(String) DEFAULT ['zzz']" diff --git a/dbms/tests/queries/0_stateless/00474_readonly_settings.sh b/dbms/tests/queries/0_stateless/00474_readonly_settings.sh index 59f83f43b40..558df9f17b2 100755 --- a/dbms/tests/queries/0_stateless/00474_readonly_settings.sh +++ b/dbms/tests/queries/0_stateless/00474_readonly_settings.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=0 | grep value $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=1 | grep value -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.' -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.' +$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.' +$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" | grep value ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" | grep value diff --git a/dbms/tests/queries/0_stateless/00492_drop_temporary_table.sql b/dbms/tests/queries/0_stateless/00492_drop_temporary_table.sql index dceb4dc2267..3edb44809e9 100644 --- a/dbms/tests/queries/0_stateless/00492_drop_temporary_table.sql +++ b/dbms/tests/queries/0_stateless/00492_drop_temporary_table.sql @@ -2,7 +2,9 @@ DROP TEMPORARY TABLE IF EXISTS temp_tab; CREATE TEMPORARY TABLE temp_tab (number UInt64); INSERT INTO temp_tab SELECT number FROM system.numbers LIMIT 1; SELECT number FROM temp_tab; +SET server_logs_level='none'; DROP TABLE temp_tab; +SET server_logs_level='warning'; CREATE TEMPORARY TABLE temp_tab (number UInt64); SELECT number FROM temp_tab; DROP TEMPORARY TABLE temp_tab; diff --git a/dbms/tests/queries/0_stateless/00550_join_insert_select.sh b/dbms/tests/queries/0_stateless/00550_join_insert_select.sh index 3e78942f989..3fbad66e7c2 100755 --- a/dbms/tests/queries/0_stateless/00550_join_insert_select.sh +++ b/dbms/tests/queries/0_stateless/00550_join_insert_select.sh @@ -20,6 +20,6 @@ INSERT INTO test.test1 SELECT id, name FROM test.test2 ANY LEFT OUTER JOIN test. DROP TABLE test.test1; DROP TABLE test.test2; DROP TABLE test.test3; -" 2>&1 | grep -F "Number of columns doesn't match" | wc -l +" --server_logs_file=/dev/null 2>&1 | grep -F "Number of columns doesn't match" | wc -l $CLICKHOUSE_CLIENT --query="SELECT 1"; diff --git a/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql b/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql index 49aff2aa184..9cef126ca49 100644 --- a/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql +++ b/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql @@ -15,5 +15,5 @@ INSERT INTO test.nums_buf (n) VALUES (5); SELECT n,m FROM test.nums ORDER BY n; SELECT n,m FROM test.nums_buf ORDER BY n; -DROP TABLE IF EXISTS test.nums; DROP TABLE IF EXISTS test.nums_buf; +DROP TABLE IF EXISTS test.nums; diff --git a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh b/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh index 7db122dcbb7..56e11138e2f 100755 --- a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh +++ b/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh @@ -2,7 +2,7 @@ [ -z "$CLICKHOUSE_CLIENT" ] && CLICKHOUSE_CLIENT="clickhouse-client" -SETTINGS="--compile=1 --min_count_to_compile=0 --max_threads=1 --max_memory_usage=8000000" +SETTINGS="--compile=1 --min_count_to_compile=0 --max_threads=1 --max_memory_usage=8000000 --server_logs_file=/dev/null" output=$($CLICKHOUSE_CLIENT -q "SELECT length(groupArray(number)) FROM (SELECT * FROM system.numbers LIMIT 1000000)" $SETTINGS 2>&1) [[ $? -eq 0 ]] && echo "Expected non-zero RC" diff --git a/dbms/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh b/dbms/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh index 08c7f8ef298..d199096815c 100755 --- a/dbms/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh +++ b/dbms/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh @@ -10,7 +10,7 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test;" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.test (dt Date DEFAULT now(), id UInt32, id2 UInt32 DEFAULT id + 1) ENGINE = MergeTree(dt, dt, 8192);" ${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test(dt,id) VALUES ('2018-02-22',3), ('2018-02-22',4), ('2018-02-22',5);" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test.test ORDER BY id;" -echo `${CLICKHOUSE_CLIENT} --query "ALTER TABLE test.test DROP COLUMN id;" 2>&1 | grep -c "$exception_pattern"` +echo `${CLICKHOUSE_CLIENT} --query "ALTER TABLE test.test DROP COLUMN id;" --server_logs_file=/dev/null 2>&1 | grep -c "$exception_pattern"` ${CLICKHOUSE_CLIENT} --query "ALTER TABLE test.test DROP COLUMN id2;" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test.test ORDER BY id;" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE test.test DROP COLUMN id;" diff --git a/dbms/tests/queries/0_stateless/00595_insert_into_view.sh b/dbms/tests/queries/0_stateless/00595_insert_into_view.sh index 030f1931928..ae3e7d40835 100755 --- a/dbms/tests/queries/0_stateless/00595_insert_into_view.sh +++ b/dbms/tests/queries/0_stateless/00595_insert_into_view.sh @@ -11,7 +11,7 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_view;" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.test (s String) ENGINE = Log;" ${CLICKHOUSE_CLIENT} --query "CREATE VIEW test.test_view AS SELECT * FROM test.test;" -echo `${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_view VALUES('test_string');" 2>&1 | grep -c "$exception_pattern"` +(( `${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_view VALUES('test_string');" 2>&1 | grep -c "$exception_pattern"` >= 1 )) && echo 1 || echo "NO MATCH" ${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test VALUES('test_string');" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test.test;" diff --git a/dbms/tests/queries/0_stateless/00596_limit_on_expanded_ast.sh b/dbms/tests/queries/0_stateless/00596_limit_on_expanded_ast.sh index 85d13cbdb47..3cbbdefb3c0 100755 --- a/dbms/tests/queries/0_stateless/00596_limit_on_expanded_ast.sh +++ b/dbms/tests/queries/0_stateless/00596_limit_on_expanded_ast.sh @@ -5,6 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) exception_pattern="too big" -${CLICKHOUSE_CLIENT} --max_expanded_ast_elements=500000 --query=" +${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --max_expanded_ast_elements=500000 --query=" select 1 as a, a+a as b, b+b as c, c+c as d, d+d as e, e+e as f, f+f as g, g+g as h, h+h as i, i+i as j, j+j as k, k+k as l, l+l as m, m+m as n, n+n as o, o+o as p, p+p as q, q+q as r, r+r as s, s+s as t, t+t as u, u+u as v, v+v as w, w+w as x, x+x as y, y+y as z " 2>&1 | grep -c "$exception_pattern" diff --git a/dbms/tests/queries/0_stateless/00602_throw_if.sh b/dbms/tests/queries/0_stateless/00602_throw_if.sh index a63c109ffaf..8dae5033978 100755 --- a/dbms/tests/queries/0_stateless/00602_throw_if.sh +++ b/dbms/tests/queries/0_stateless/00602_throw_if.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) exception_pattern="Value passed to 'throwIf' function is non zero" -${CLICKHOUSE_CLIENT} --query="SELECT throwIf(number = 1000000) FROM system.numbers" 2>&1 | grep -cF "$exception_pattern" -${CLICKHOUSE_CLIENT} --query="SELECT sum(x = 0) FROM (SELECT throwIf(number = 1000000) AS x FROM numbers(1000000))" 2>&1 +${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000) FROM system.numbers" 2>&1 | grep -cF "$exception_pattern" +${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT sum(x = 0) FROM (SELECT throwIf(number = 1000000) AS x FROM numbers(1000000))" 2>&1 diff --git a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table.sql b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table.sql index daf9950e9d5..397b459142a 100644 --- a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table.sql +++ b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table.sql @@ -43,7 +43,7 @@ SELECT count(), sum(d) FROM test.dst; INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst' AND active AND name LIKE '1_%'; SELECT (max(m) - min(m) > 1) AS new_block_is_generated FROM test_block_numbers; -DROP TABLE test_block_numbers; +DROP TEMPORARY TABLE test_block_numbers; SELECT 'ATTACH FROM'; diff --git a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sql b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sql index 304e43d497d..352bf225979 100644 --- a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sql @@ -55,7 +55,7 @@ SELECT count(), sum(d) FROM test.dst_r2; INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst_r1' AND active AND name LIKE '1_%'; SELECT (max(m) - min(m) > 1) AS new_block_is_generated FROM test_block_numbers; -DROP TABLE test_block_numbers; +DROP TEMPORARY TABLE test_block_numbers; SELECT 'ATTACH FROM'; diff --git a/dbms/tests/queries/shell_config.sh b/dbms/tests/queries/shell_config.sh index 0c12418b602..00b7b67c50a 100644 --- a/dbms/tests/queries/shell_config.sh +++ b/dbms/tests/queries/shell_config.sh @@ -1,6 +1,6 @@ export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"} -export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="${CLICKHOUSE_BINARY}-client"} +export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="${CLICKHOUSE_BINARY}-client --server_logs_level=warning"} export CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"} export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:="/etc/clickhouse-server/config.xml"} From c04dfb40a62556b77e97acca2f98440a4f514c59 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Sat, 9 Jun 2018 18:29:08 +0300 Subject: [PATCH 079/315] Fixed memory tracking. [#CLICKHOUSE-2910] --- dbms/src/Client/Connection.cpp | 2 +- dbms/src/Common/MemoryTracker.cpp | 44 ++++++++++++------- dbms/src/Common/MemoryTracker.h | 14 ++++-- dbms/src/Common/ProfileEvents.cpp | 4 +- dbms/src/Common/ProfileEvents.h | 16 ++----- dbms/src/Common/ThreadStatus.cpp | 14 ++++-- dbms/src/Common/ThreadStatus.h | 15 ++++--- dbms/src/Common/VariableContext.h | 12 +++++ dbms/src/Interpreters/ProcessList.cpp | 14 ++---- dbms/src/Interpreters/ProcessList.h | 10 ++--- dbms/src/Interpreters/QueryThreadLog.cpp | 4 +- dbms/src/Interpreters/QueryThreadLog.h | 2 +- dbms/src/Server/Client.cpp | 2 +- dbms/src/Storages/MergeTree/MergeList.h | 2 +- .../00443_optimize_final_vertical_merge.sh | 2 +- 15 files changed, 91 insertions(+), 66 deletions(-) create mode 100644 dbms/src/Common/VariableContext.h diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index cbd588a25d1..59285d3b03f 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -521,7 +521,7 @@ Connection::Packet Connection::receivePacket() } else { - LOG_TRACE(log_wrapper.get(), "Receiving packet type"); + //LOG_TRACE(log_wrapper.get(), "Receiving packet type " << StackTrace().toString()); readVarUInt(res.type, *in); } diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index dc8f86c78f7..72f03d17829 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -18,7 +18,7 @@ namespace DB MemoryTracker::~MemoryTracker() { - if (peak) + if (level != VariableContext::Thread && peak) { try { @@ -114,18 +114,26 @@ void MemoryTracker::free(Int64 size) if (blocker.isCancelled()) return; - Int64 new_amount = amount.fetch_sub(size, std::memory_order_relaxed) - size; - - /** Sometimes, query could free some data, that was allocated outside of query context. - * Example: cache eviction. - * To avoid negative memory usage, we "saturate" amount. - * Memory usage will be calculated with some error. - * NOTE The code is not atomic. Not worth to fix. - */ - if (new_amount < 0) + if (level == VariableContext::Thread) { - amount.fetch_sub(new_amount); - size += new_amount; + /// Could become negative if memory allocated in this thread is freed in another one + amount.fetch_sub(size, std::memory_order_relaxed); + } + else + { + Int64 new_amount = amount.fetch_sub(size, std::memory_order_relaxed) - size; + + /** Sometimes, query could free some data, that was allocated outside of query context. + * Example: cache eviction. + * To avoid negative memory usage, we "saturate" amount. + * Memory usage will be calculated with some error. + * NOTE: The code is not atomic. Not worth to fix. + */ + if (unlikely(new_amount < 0)) + { + amount.fetch_sub(new_amount); + size += new_amount; + } } if (auto loaded_next = parent.load(std::memory_order_relaxed)) @@ -135,14 +143,20 @@ void MemoryTracker::free(Int64 size) } +void MemoryTracker::resetCounters() +{ + amount.store(0, std::memory_order_relaxed); + peak.store(0, std::memory_order_relaxed); + limit.store(0, std::memory_order_relaxed); +} + + void MemoryTracker::reset() { if (!parent.load(std::memory_order_relaxed)) CurrentMetrics::sub(metric, amount.load(std::memory_order_relaxed)); - amount.store(0, std::memory_order_relaxed); - peak.store(0, std::memory_order_relaxed); - limit.store(0, std::memory_order_relaxed); + resetCounters(); } diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index 63e21da6ac1..bf6a0a42bc0 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -1,5 +1,6 @@ #pragma once +#include "VariableContext.h" #include #include #include @@ -36,12 +37,14 @@ class MemoryTracker const char * description = nullptr; public: - MemoryTracker() {} - MemoryTracker(Int64 limit_) : limit(limit_) {} - MemoryTracker(MemoryTracker * parent_) : parent(parent_) {} + MemoryTracker(VariableContext level = VariableContext::Thread) : level(level) {} + MemoryTracker(Int64 limit_, VariableContext level = VariableContext::Thread) : limit(limit_), level(level) {} + MemoryTracker(MemoryTracker * parent_, VariableContext level = VariableContext::Thread) : parent(parent_), level(level) {} ~MemoryTracker(); + VariableContext level; + /** Call the following functions before calling of corresponding operations with memory allocators. */ void alloc(Int64 size); @@ -103,7 +106,10 @@ public: description = description_; } - /// Reset the accumulated data. + /// Reset the accumulated data + void resetCounters(); + + /// Reset the accumulated data and the parent. void reset(); /// Prints info about peak memory consumption into log. diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 976f8976547..020601797f1 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -181,7 +181,7 @@ Counters global_counters(global_counters_array); const Event Counters::num_counters = END; -Counters::Counters(Level level, Counters * parent) +Counters::Counters(VariableContext level, Counters * parent) : counters_holder(new Counter[num_counters] {}), parent(parent), level(level) @@ -206,7 +206,7 @@ void Counters::reset() Counters Counters::getPartiallyAtomicSnapshot() const { - Counters res(Level::Snapshot, nullptr); + Counters res(VariableContext::Snapshot, nullptr); for (Event i = 0; i < num_counters; ++i) res.counters[i].store(counters[i].load(std::memory_order_relaxed), std::memory_order_relaxed); return res; diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index ed69c8d78ac..ef18ef5531a 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -1,5 +1,6 @@ #pragma once +#include "VariableContext.h" #include #include #include @@ -27,15 +28,6 @@ namespace ProfileEvents /// Counters - how many times each event happened extern Counters global_counters; - enum class Level - { - Global = 0, - User, - Process, - Thread, - Snapshot - }; - class Counters { Counter * counters = nullptr; @@ -45,14 +37,14 @@ namespace ProfileEvents public: - Level level = Level::Thread; + VariableContext level = VariableContext::Thread; /// By default, any instance have to increment global counters - Counters(Level level = Level::Thread, Counters * parent = &global_counters); + Counters(VariableContext level = VariableContext::Thread, Counters * parent = &global_counters); /// Global level static initializer Counters(Counter * allocated_counters) - : counters(allocated_counters), parent(nullptr), level(Level::Global) {} + : counters(allocated_counters), parent(nullptr), level(VariableContext::Global) {} inline Counter & operator[] (Event event) { diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index a117d9ed692..abaaa12da3e 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -58,6 +58,7 @@ public: ThreadStatus & thread = *CurrentThread::get(); LOG_DEBUG(thread.log, "Thread " << thread.thread_number << " exited"); + thread.memory_tracker.logPeakMemoryUsage(); if (thread.getCurrentState() != ThreadStatus::ThreadState::DetachedFromQuery) thread.detachQuery(true); @@ -192,14 +193,12 @@ struct TasksStatsCounters TasksStatsCounters TasksStatsCounters::current() { TasksStatsCounters res; - current_thread->taskstats_getter->getStat(res.stat); + current_thread->taskstats_getter->getStat(res.stat, current_thread->os_thread_id); return res; } ThreadStatus::ThreadStatus() - : performance_counters(ProfileEvents::Level::Thread), - log(&Poco::Logger::get("ThreadStatus")) { thread_number = Poco::ThreadNumber::get(); os_thread_id = TaskStatsInfoGetter::getCurrentTID(); @@ -208,6 +207,9 @@ ThreadStatus::ThreadStatus() last_taskstats = std::make_unique(); taskstats_getter = std::make_unique(); + memory_tracker.setDescription("(for thread)"); + log = &Poco::Logger::get("ThreadStatus"); + /// NOTE: It is important not to do any non-trivial actions (like updating ProfileEvents or logging) before ThreadStatus is created /// Otherwise it could lead to SIGSEGV due to current_thread dereferencing } @@ -287,7 +289,10 @@ void ThreadStatus::attachQuery( /// Clear stats from previous query if a new query is started /// TODO: make separate query_thread_performance_counters and thread_performance_counters if (queries_started != 1) + { performance_counters.resetCounters(); + memory_tracker.resetCounters(); + } *last_rusage = RusageCounters::current(query_start_time_nanoseconds); *last_taskstats = TasksStatsCounters::current(); @@ -327,6 +332,7 @@ void ThreadStatus::detachQuery(bool thread_exits) log_profile_events = true; } + void ThreadStatus::updatePerfomanceCountersImpl() { try @@ -352,7 +358,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) elem.read_bytes = progress_in.bytes.load(std::memory_order_relaxed); elem.written_rows = progress_out.rows.load(std::memory_order_relaxed); elem.written_bytes = progress_out.bytes.load(std::memory_order_relaxed); - elem.memory_usage = std::max(0, memory_tracker.getPeak()); + elem.memory_usage = memory_tracker.getPeak(); elem.thread_name = getThreadName(); elem.thread_number = thread_number; diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 7d920597191..f5ede8593a9 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -40,8 +40,9 @@ public: /// Linux's PID (or TGID) (the same id is shown by ps util) Int32 os_thread_id = -1; - ProfileEvents::Counters performance_counters; - MemoryTracker memory_tracker; + /// TODO: merge them into common entity + ProfileEvents::Counters performance_counters{VariableContext::Thread}; + MemoryTracker memory_tracker{VariableContext::Thread}; /// Statistics of read and write rows/bytes Progress progress_in; @@ -67,11 +68,6 @@ public: return thread_state.load(std::memory_order_relaxed); } - Context * getGlobalContext() - { - return global_context.load(std::memory_order_relaxed); - } - SystemLogsQueuePtr getSystemLogsQueue() const { std::lock_guard lock(mutex); @@ -84,6 +80,11 @@ public: logs_queue_ptr = logs_queue; } + Context * getGlobalContext() + { + return global_context.load(std::memory_order_relaxed); + } + ~ThreadStatus(); protected: diff --git a/dbms/src/Common/VariableContext.h b/dbms/src/Common/VariableContext.h new file mode 100644 index 00000000000..2fe4ffb565a --- /dev/null +++ b/dbms/src/Common/VariableContext.h @@ -0,0 +1,12 @@ +#pragma once + +/// Used in ProfileEvents and MemoryTracker to determine their hierarchy level +/// The less value the higher level (zero level is the root) +enum class VariableContext +{ + Global = 0, + User, /// Group of processes + Process, /// For example, a query or a merge + Thread, /// A thread of a process + Snapshot /// Does not belong to anybody +}; diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index b8595d604ac..0e55ccc4642 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -168,8 +168,8 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as /// Attach master thread CurrentThread::attachQuery(&*process_it); - CurrentThread::getMemoryTracker().setOrRaiseLimit(settings.max_memory_usage); - CurrentThread::getMemoryTracker().setDescription("(for thread)"); + /// 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 if (!user_process_list.user_throttler) { @@ -272,7 +272,6 @@ QueryStatus::QueryStatus( query(query_), client_info(client_info_), priority_handle(std::move(priority_handle_)), - performance_counters(ProfileEvents::Level::Process), num_queries_increment{CurrentMetrics::Query} { memory_tracker.setOrRaiseLimit(max_memory_usage); @@ -282,10 +281,7 @@ QueryStatus::QueryStatus( memory_tracker.setFaultProbability(memory_tracker_fault_probability); } -QueryStatus::~QueryStatus() -{ - LOG_DEBUG(&Poco::Logger::get("QueryStatus"), __PRETTY_FUNCTION__ << ":" << __LINE__); -} +QueryStatus::~QueryStatus() = default; void QueryStatus::setQueryStreams(const BlockIO & io) { @@ -444,9 +440,7 @@ ProcessList::Info ProcessList::getInfo(bool get_thread_list, bool get_profile_ev } -ProcessListForUser::ProcessListForUser() -: user_performance_counters(ProfileEvents::Level::User, &ProfileEvents::global_counters) -{} +ProcessListForUser::ProcessListForUser() = default; } diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index df1db61b498..584143380b4 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -96,8 +96,8 @@ protected: QueryPriorities::Handle priority_handle; - ProfileEvents::Counters performance_counters; - MemoryTracker memory_tracker; + ProfileEvents::Counters performance_counters{VariableContext::Process}; + MemoryTracker memory_tracker{VariableContext::Process}; mutable std::shared_mutex threads_mutex; /// Key is Poco's thread_id @@ -207,9 +207,9 @@ struct ProcessListForUser using QueryToElement = std::unordered_multimap; QueryToElement queries; - ProfileEvents::Counters user_performance_counters; + ProfileEvents::Counters user_performance_counters{VariableContext::User, &ProfileEvents::global_counters}; /// Limit and counter for memory of all simultaneously running queries of single user. - MemoryTracker user_memory_tracker; + MemoryTracker user_memory_tracker{VariableContext::User}; /// Count network usage for all simultaneously running queries of single user. ThrottlerPtr user_throttler; @@ -282,7 +282,7 @@ protected: QueryPriorities priorities; /// Limit and counter for memory of all simultaneously running queries. - MemoryTracker total_memory_tracker; + MemoryTracker total_memory_tracker{VariableContext::Global}; /// Limit network bandwidth for all users ThrottlerPtr total_network_throttler; diff --git a/dbms/src/Interpreters/QueryThreadLog.cpp b/dbms/src/Interpreters/QueryThreadLog.cpp index dc290917f56..7b9dbe02d83 100644 --- a/dbms/src/Interpreters/QueryThreadLog.cpp +++ b/dbms/src/Interpreters/QueryThreadLog.cpp @@ -30,7 +30,7 @@ Block QueryThreadLogElement::createBlock() {std::make_shared(), "read_bytes"}, {std::make_shared(), "written_rows"}, {std::make_shared(), "written_bytes"}, - {std::make_shared(), "memory_usage"}, + {std::make_shared(), "memory_usage"}, {std::make_shared(), "thread_name"}, {std::make_shared(), "thread_number"}, @@ -80,7 +80,7 @@ void QueryThreadLogElement::appendToBlock(Block & block) const columns[i++]->insert(UInt64(written_rows)); columns[i++]->insert(UInt64(written_bytes)); - columns[i++]->insert(UInt64(memory_usage)); + columns[i++]->insert(Int64(memory_usage)); columns[i++]->insertData(thread_name.data(), thread_name.size()); columns[i++]->insert(UInt64(thread_number)); diff --git a/dbms/src/Interpreters/QueryThreadLog.h b/dbms/src/Interpreters/QueryThreadLog.h index 3552f15c623..9de116fa8b6 100644 --- a/dbms/src/Interpreters/QueryThreadLog.h +++ b/dbms/src/Interpreters/QueryThreadLog.h @@ -22,7 +22,7 @@ struct QueryThreadLogElement UInt64 written_rows{}; UInt64 written_bytes{}; - UInt64 memory_usage{}; + Int64 memory_usage{}; String thread_name; UInt32 thread_number{}; diff --git a/dbms/src/Server/Client.cpp b/dbms/src/Server/Client.cpp index b013d195209..5b251355e88 100644 --- a/dbms/src/Server/Client.cpp +++ b/dbms/src/Server/Client.cpp @@ -1529,7 +1529,7 @@ public: ("echo", "in batch mode, print query before execution") ("max_client_network_bandwidth", po::value(), "the maximum speed of data exchange over the network for the client in bytes per second.") ("compression", po::value(), "enable or disable compression") - ("log-level", po::value(), "log level") + ("log-level", po::value(), "client log level") ("server_logs_file", po::value(), "put server logs into specified file") APPLY_FOR_SETTINGS(DECLARE_SETTING) ; diff --git a/dbms/src/Storages/MergeTree/MergeList.h b/dbms/src/Storages/MergeTree/MergeList.h index 464cca89812..7bc98a02610 100644 --- a/dbms/src/Storages/MergeTree/MergeList.h +++ b/dbms/src/Storages/MergeTree/MergeList.h @@ -65,7 +65,7 @@ struct MergeListElement : boost::noncopyable /// Updated only for Vertical algorithm std::atomic columns_written{}; - MemoryTracker memory_tracker; + MemoryTracker memory_tracker{VariableContext::Process}; MemoryTracker * background_thread_memory_tracker; MemoryTracker * background_thread_memory_tracker_prev_parent = nullptr; diff --git a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh index 43392107b31..c8453119eb9 100755 --- a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh +++ b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh @@ -70,7 +70,7 @@ number AS di10, [hex(number), hex(number+1)] AS \`n.s\` FROM system.numbers LIMIT $res_rows" -while [[ `get_num_parts` -ne 1 ]] ; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001"; done +while [[ `get_num_parts` -ne 1 ]] ; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001" --server_logs_file=/dev/null; done $CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN n.a Array(String)" $CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN da Array(String) DEFAULT ['def']" From 9a788d97907a5a23a5bf7309bfe6630207b89d13 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Sat, 9 Jun 2018 18:52:59 +0300 Subject: [PATCH 080/315] Fixed exception safety in TCPHandler. [#CLICKHOUSE-2910] --- dbms/src/Client/Connection.cpp | 2 +- dbms/src/Server/TCPHandler.cpp | 12 +++++++++++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 59285d3b03f..6c18a262ebb 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -521,7 +521,7 @@ Connection::Packet Connection::receivePacket() } else { - //LOG_TRACE(log_wrapper.get(), "Receiving packet type " << StackTrace().toString()); + //LOG_TRACE(log_wrapper.get(), "Receiving packet type"); readVarUInt(res.type, *in); } diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index b37e38c696a..9555e95de76 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -241,7 +241,17 @@ void TCPHandler::runImpl() { if (exception) { - sendLogs(); + try + { + /// Try to send logs to client, but it could be risky too + /// Assume that we can't break output here + sendLogs(); + } + catch (...) + { + tryLogCurrentException(log, "Can't send logs to client"); + } + sendException(*exception); } } From 95b42afcb9519086de828b8b1beafec6752690af Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Wed, 13 Jun 2018 22:01:07 +0300 Subject: [PATCH 081/315] Add test for profile events logging. [#CLICKHOUSE-2910] --- dbms/src/Client/Connection.cpp | 3 +- dbms/src/Common/CurrentThread.cpp | 5 + dbms/src/Common/CurrentThread.h | 1 + dbms/src/Common/ThreadStatus.cpp | 31 +++-- dbms/src/Common/ThreadStatus.h | 2 +- dbms/src/Common/setThreadName.cpp | 3 +- .../AsynchronousBlockInputStream.cpp | 80 +++++++++++++ .../DataStreams/RemoteBlockOutputStream.cpp | 10 +- .../src/Interpreters/ClickHouseLogChannel.cpp | 1 + dbms/src/Interpreters/ClickHouseLogChannel.h | 17 +++ dbms/src/Interpreters/Context.cpp | 17 ++- dbms/src/Interpreters/Context.h | 6 +- .../Interpreters/InterpreterSystemQuery.cpp | 32 ++++-- dbms/src/Interpreters/ProcessList.cpp | 5 +- dbms/src/Interpreters/SystemLog.h | 7 +- dbms/src/Interpreters/executeQuery.cpp | 6 + dbms/src/Parsers/ASTSystemQuery.cpp | 2 + dbms/src/Parsers/ASTSystemQuery.h | 1 + dbms/src/Server/Client.cpp | 15 ++- dbms/src/Server/TCPHandler.cpp | 6 +- dbms/tests/clickhouse-test | 17 +-- .../00534_long_functions_bad_arguments.lib | 3 - ...rmance_introspection_and_logging.reference | 5 + ...4_performance_introspection_and_logging.sh | 106 ++++++++++++++++++ dbms/tests/queries/shell_config.sh | 4 +- 25 files changed, 321 insertions(+), 64 deletions(-) create mode 100644 dbms/src/DataStreams/AsynchronousBlockInputStream.cpp create mode 100644 dbms/src/Interpreters/ClickHouseLogChannel.h create mode 100644 dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.reference create mode 100755 dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 6c18a262ebb..213723beae1 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -514,6 +514,7 @@ Connection::Packet Connection::receivePacket() { Packet res; + /// Have we already read packet type? if (last_input_packet_type) { res.type = *last_input_packet_type; @@ -525,7 +526,7 @@ Connection::Packet Connection::receivePacket() readVarUInt(res.type, *in); } - // LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type)); + //LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type)); switch (res.type) { diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 620a3a52738..5abc4888275 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -71,6 +71,11 @@ void CurrentThread::detachQuery() getCurrentThreadImpl()->detachQuery(); } +void CurrentThread::detachQueryIfNotDetached() +{ + getCurrentThreadImpl()->detachQuery(true); +} + ProfileEvents::Counters & CurrentThread::getProfileEvents() { return current_thread->performance_counters; diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index 1d9ed769dc6..5ddd2b0fcee 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -52,6 +52,7 @@ public: /// Non-master threads call this method in destructor automatically static void detachQuery(); + static void detachQueryIfNotDetached(); private: static void attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thread, bool check_detached = true); diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index abaaa12da3e..37d464f3a04 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -58,12 +58,7 @@ public: ThreadStatus & thread = *CurrentThread::get(); LOG_DEBUG(thread.log, "Thread " << thread.thread_number << " exited"); - thread.memory_tracker.logPeakMemoryUsage(); - - if (thread.getCurrentState() != ThreadStatus::ThreadState::DetachedFromQuery) - thread.detachQuery(true); - else - thread.thread_state = ThreadStatus::ThreadState::Died; + thread.detachQuery(true, true); } catch (...) { @@ -251,10 +246,15 @@ void ThreadStatus::attachQuery( parent_query = parent_query_; performance_counters.setParent(parent_counters); memory_tracker.setParent(parent_memory_tracker); - memory_tracker.setDescription("(for thread)"); logs_queue_ptr = logs_queue_ptr_; } + /// Clear stats from previous query if a new query is started + /// TODO: make separate query_thread_performance_counters and thread_performance_counters + performance_counters.resetCounters(); + memory_tracker.resetCounters(); + memory_tracker.setDescription("(for thread)"); + /// Try extract as many information as possible from ProcessList if (parent_query) { @@ -286,20 +286,18 @@ void ThreadStatus::attachQuery( query_start_time = time(nullptr); ++queries_started; - /// Clear stats from previous query if a new query is started - /// TODO: make separate query_thread_performance_counters and thread_performance_counters - if (queries_started != 1) - { - performance_counters.resetCounters(); - memory_tracker.resetCounters(); - } - *last_rusage = RusageCounters::current(query_start_time_nanoseconds); *last_taskstats = TasksStatsCounters::current(); } -void ThreadStatus::detachQuery(bool thread_exits) +void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) { + if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery) + { + thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; + return; + } + if (thread_state != ThreadState::AttachedToQuery && thread_state != ThreadState::QueryInitializing) throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); @@ -327,7 +325,6 @@ void ThreadStatus::detachQuery(bool thread_exits) } thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; - log_to_query_thread_log = true; log_profile_events = true; } diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index f5ede8593a9..8d846202e04 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -100,7 +100,7 @@ protected: const SystemLogsQueueWeakPtr & logs_queue_ptr_, bool check_detached = true); - void detachQuery(bool thread_exits = false); + void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false); void logToQueryThreadLog(QueryThreadLog & thread_log); diff --git a/dbms/src/Common/setThreadName.cpp b/dbms/src/Common/setThreadName.cpp index a151b1952f1..d92b2e84715 100644 --- a/dbms/src/Common/setThreadName.cpp +++ b/dbms/src/Common/setThreadName.cpp @@ -7,6 +7,7 @@ #include #endif #include +#include #include #include @@ -46,6 +47,6 @@ std::string getThreadName() #endif DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)"); - name.resize(strlen(name.data())); + name.resize(std::strlen(name.data())); return name; } diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp new file mode 100644 index 00000000000..25055001ecc --- /dev/null +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp @@ -0,0 +1,80 @@ +#include "AsynchronousBlockInputStream.h" +#include + + +namespace DB +{ + +Block AsynchronousBlockInputStream::readImpl() +{ + /// If there were no calculations yet, calculate the first block synchronously + if (!started) + { + calculate(); + started = true; + } + else /// If the calculations are already in progress - wait for the result + pool.wait(); + + if (exception) + std::rethrow_exception(exception); + + Block res = block; + if (!res) + return res; + + /// Start the next block calculation + block.clear(); + next(); + + return res; +} + + +void AsynchronousBlockInputStream::next() +{ + ready.reset(); + + pool.schedule([this, main_thread=CurrentThread::get()] () { + CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; + + try + { + if (first) + setThreadName("AsyncBlockInput"); + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + } + catch (...) + { + exception = std::current_exception(); + ready.set(); + return; + } + + calculate(); + }); +} + + +void AsynchronousBlockInputStream::calculate() +{ + try + { + if (first) + { + first = false; + children.back()->readPrefix(); + } + + block = children.back()->read(); + } + catch (...) + { + exception = std::current_exception(); + } + + ready.set(); +} + +} + diff --git a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp index 9bc0fa3b101..4848483253a 100644 --- a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp @@ -63,15 +63,11 @@ void RemoteBlockOutputStream::write(const Block & block) catch (const NetException & e) { /// Try to get more detailed exception from server - if (connection.poll(0)) + auto packet_type = connection.checkPacket(); + if (packet_type && *packet_type == Protocol::Server::Exception) { Connection::Packet packet = connection.receivePacket(); - - if (Protocol::Server::Exception == packet.type) - { - packet.exception->rethrow(); - return; - } + packet.exception->rethrow(); } throw; diff --git a/dbms/src/Interpreters/ClickHouseLogChannel.cpp b/dbms/src/Interpreters/ClickHouseLogChannel.cpp index 0fc15550f1d..ea69d2dc088 100644 --- a/dbms/src/Interpreters/ClickHouseLogChannel.cpp +++ b/dbms/src/Interpreters/ClickHouseLogChannel.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include diff --git a/dbms/src/Interpreters/ClickHouseLogChannel.h b/dbms/src/Interpreters/ClickHouseLogChannel.h new file mode 100644 index 00000000000..e1104476a05 --- /dev/null +++ b/dbms/src/Interpreters/ClickHouseLogChannel.h @@ -0,0 +1,17 @@ +#pragma once +#include + + +namespace DB +{ + +/// Poco::Channel used to implement passing of query logs to Client via TCP interface +class ClickHouseLogChannel : public Poco::Channel +{ +public: + ClickHouseLogChannel() = default; + + void log(const Poco::Message & msg) override; +}; + +} diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 6ac18cfbc0b..89a04432e9c 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -1499,7 +1499,7 @@ void Context::initializeSystemLogs() } -QueryLog * Context::getQueryLog() +QueryLog * Context::getQueryLog(bool create_if_not_exists) { auto lock = getLock(); @@ -1508,6 +1508,9 @@ QueryLog * Context::getQueryLog() if (!system_logs->query_log) { + if (!create_if_not_exists) + return nullptr; + if (shared->shutdown_called) throw Exception("Logical error: query log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR); @@ -1521,7 +1524,7 @@ QueryLog * Context::getQueryLog() } -QueryThreadLog * Context::getQueryThreadLog() +QueryThreadLog * Context::getQueryThreadLog(bool create_if_not_exists) { auto lock = getLock(); @@ -1530,6 +1533,9 @@ QueryThreadLog * Context::getQueryThreadLog() if (!system_logs->query_thread_log) { + if (!create_if_not_exists) + return nullptr; + if (shared->shutdown_called) throw Exception("Logical error: query log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR); @@ -1544,7 +1550,7 @@ QueryThreadLog * Context::getQueryThreadLog() } -PartLog * Context::getPartLog(const String & part_database) +PartLog * Context::getPartLog(const String & part_database, bool create_if_not_exists) { auto lock = getLock(); @@ -1561,11 +1567,14 @@ PartLog * Context::getPartLog(const String & part_database) /// Will not log operations on system tables (including part_log itself). /// It doesn't make sense and not allow to destruct PartLog correctly due to infinite logging and flushing, /// and also make troubles on startup. - if (part_database == database) + if (!part_database.empty() && part_database == database) return nullptr; if (!system_logs->part_log) { + if (!create_if_not_exists) + return nullptr; + if (shared->shutdown_called) throw Exception("Logical error: part log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 36b2e1e2905..3005d023b93 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -361,12 +361,12 @@ public: void initializeSystemLogs(); /// Nullptr if the query log is not ready for this moment. - QueryLog * getQueryLog(); - QueryThreadLog * getQueryThreadLog(); + QueryLog * getQueryLog(bool create_if_not_exists = true); + QueryThreadLog * getQueryThreadLog(bool create_if_not_exists = true); /// Returns an object used to log opertaions with parts if it possible. /// Provide table name to make required cheks. - PartLog * getPartLog(const String & part_database); + PartLog * getPartLog(const String & part_database, bool create_if_not_exists = true); const MergeTreeSettings & getMergeTreeSettings(); diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 46651a12fae..ace52b26184 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -10,6 +10,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -48,7 +51,7 @@ ExecutionStatus getOverallExecutionStatusOfCommands() return ExecutionStatus(0); } -/// Consequently execute all commands and genreates final exception message for failed commands +/// Consequently tries to execute all commands and genreates final exception message for failed commands template ExecutionStatus getOverallExecutionStatusOfCommands(Callable && command, Callables && ... commands) { @@ -70,6 +73,16 @@ ExecutionStatus getOverallExecutionStatusOfCommands(Callable && command, Callabl return ExecutionStatus(res_status, res_message); } +/// Consequently tries to execute all commands and throws exception with info about failed commands +template +void executeCommandsAndThrowIfError(Callables && ... commands) +{ + auto status = getOverallExecutionStatusOfCommands(std::forward(commands)...); + if (status.code != 0) + throw Exception(status.message, status.code); +} + + /// Implements SYSTEM [START|STOP] void startStopAction(Context & context, ASTSystemQuery & query, StorageActionBlockType action_type, bool start) { @@ -139,15 +152,11 @@ BlockIO InterpreterSystemQuery::execute() system_context.getExternalDictionaries().reloadDictionary(query.target_dictionary); break; case Type::RELOAD_DICTIONARIES: - { - auto status = getOverallExecutionStatusOfCommands( - [&] { system_context.getExternalDictionaries().reload(); }, - [&] { system_context.getEmbeddedDictionaries().reload(); } + executeCommandsAndThrowIfError( + [&] () { system_context.getExternalDictionaries().reload(); }, + [&] () { system_context.getEmbeddedDictionaries().reload(); } ); - if (status.code != 0) - throw Exception(status.message, status.code); break; - } case Type::RELOAD_EMBEDDED_DICTIONARIES: system_context.getEmbeddedDictionaries().reload(); break; @@ -189,6 +198,13 @@ BlockIO InterpreterSystemQuery::execute() throw Exception("There is no " + query.target_database + "." + query.target_table + " replicated table", ErrorCodes::BAD_ARGUMENTS); break; + case Type::FLUSH_SYSTEM_TABLES: + executeCommandsAndThrowIfError( + [&] () { if (auto query_log = context.getQueryLog(false)) query_log->flush(); }, + [&] () { if (auto part_log = context.getPartLog("", false)) part_log->flush(); }, + [&] () { if (auto query_thread_log = context.getQueryThreadLog(false)) query_thread_log->flush(); } + ); + break; case Type::STOP_LISTEN_QUERIES: case Type::START_LISTEN_QUERIES: throw Exception(String(ASTSystemQuery::typeToString(query.type)) + " is not supported yet", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 0e55ccc4642..f96032952d6 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -167,7 +167,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as /// Query-level memory tracker is already set in the QueryStatus constructor /// Attach master thread - CurrentThread::attachQuery(&*process_it); + CurrentThread::attachQuery(&(*process_it)); /// 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 @@ -196,9 +196,8 @@ ProcessListEntry::~ProcessListEntry() it->releaseQueryStreams(); /// Finalize all threads statuses + CurrentThread::detachQueryIfNotDetached(); { - CurrentThread::detachQuery(); - std::shared_lock lock(it->threads_mutex); for (auto & elem : it->thread_statuses) elem.second->clean(); diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index 584146b0de5..2b0e8a71089 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -103,6 +103,9 @@ public: LOG_ERROR(log, "SystemLog queue is full"); } + /// Flush data in the buffer to disk + void flush(); + protected: Context & context; const String database_name; @@ -110,6 +113,7 @@ protected: const String storage_def; StoragePtr table; const size_t flush_interval_milliseconds; + std::mutex flush_mutex; using QueueItem = std::pair; /// First element is shutdown flag for thread. @@ -129,7 +133,6 @@ protected: std::thread saving_thread; void threadFunction(); - void flush(); /** Creates new table if it does not exist. * Renames old table if its structure is not suitable. @@ -233,6 +236,8 @@ void SystemLog::threadFunction() template void SystemLog::flush() { + std::lock_guard flush_lock(flush_mutex); + try { LOG_TRACE(log, "Flushing system log"); diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index fc48e1a8ba6..7381b438ae7 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -278,6 +278,9 @@ static std::tuple executeQueryImpl( if (!process_list_elem) return; + /// Update performance counters before logging to query_log + CurrentThread::detachQuery(); + QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events); double elapsed_seconds = info.elapsed_seconds; @@ -348,6 +351,9 @@ static std::tuple executeQueryImpl( QueryStatus * process_list_elem = context.getProcessListElement(); const Settings & settings = context.getSettingsRef(); + /// Update performance counters before logging to query_log + CurrentThread::detachQuery(); + if (process_list_elem) { QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events); diff --git a/dbms/src/Parsers/ASTSystemQuery.cpp b/dbms/src/Parsers/ASTSystemQuery.cpp index 01f43e3d909..51bd2c8c26a 100644 --- a/dbms/src/Parsers/ASTSystemQuery.cpp +++ b/dbms/src/Parsers/ASTSystemQuery.cpp @@ -61,6 +61,8 @@ const char * ASTSystemQuery::typeToString(Type type) return "STOP REPLICATION QUEUES"; case Type::START_REPLICATION_QUEUES: return "START REPLICATION QUEUES"; + case Type::FLUSH_SYSTEM_TABLES: + return "FLUSH SYSTEM TABLES"; default: throw Exception("Unknown SYSTEM query command", ErrorCodes::BAD_TYPE_OF_FIELD); } diff --git a/dbms/src/Parsers/ASTSystemQuery.h b/dbms/src/Parsers/ASTSystemQuery.h index b3366f7f4da..bd772421ec8 100644 --- a/dbms/src/Parsers/ASTSystemQuery.h +++ b/dbms/src/Parsers/ASTSystemQuery.h @@ -35,6 +35,7 @@ public: START_REPLICATEDS_SENDS, STOP_REPLICATION_QUEUES, START_REPLICATION_QUEUES, + FLUSH_SYSTEM_TABLES, END }; diff --git a/dbms/src/Server/Client.cpp b/dbms/src/Server/Client.cpp index 5b251355e88..3a4745a6b4c 100644 --- a/dbms/src/Server/Client.cpp +++ b/dbms/src/Server/Client.cpp @@ -372,7 +372,12 @@ private: format_max_block_size = config().getInt("format_max_block_size", context.getSettingsRef().max_block_size); insert_format = "Values"; - insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size); + + /// Setting value from cmd arg overrides one from config + if (context.getSettingsRef().max_insert_block_size.changed) + insert_format_max_block_size = context.getSettingsRef().max_insert_block_size; + else + insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size); if (!is_interactive) { @@ -957,13 +962,13 @@ private: connection->sendData(block); processed_rows += block.rows(); - if (!block) - break; - /// Check if server send Log packet auto packet_type = connection->checkPacket(); if (packet_type && *packet_type == Protocol::Server::Log) - connection->receivePacket(); + receiveAndProcessPacket(); + + if (!block) + break; } async_block_input->readSuffix(); diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 9555e95de76..8d635181dae 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -296,12 +296,12 @@ void TCPHandler::readData(const Settings & global_settings) constexpr size_t min_poll_interval = 5000; // 5 ms size_t poll_interval = std::max(min_poll_interval, std::min(default_poll_interval, current_poll_interval)); + sendLogs(); + while (true) { Stopwatch watch(CLOCK_MONOTONIC_COARSE); - sendLogs(); - /// We are waiting for a packet from the client. Thus, every `POLL_INTERVAL` seconds check whether we need to shut down. while (true) { @@ -335,6 +335,8 @@ void TCPHandler::readData(const Settings & global_settings) /// We accept and process data. And if they are over, then we leave. if (!receivePacket()) break; + + sendLogs(); } } diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 3ab93acda99..a8d257aabf1 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -79,6 +79,11 @@ def main(args): os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) os.environ.setdefault("CLICKHOUSE_TMP", tmp_dir) + # Force to print server warnings in stderr + # Sell scripts could change logging level + server_logs_level="warning" + os.environ.setdefault("CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL", server_logs_level) + # TODO ! use clickhouse-extract-from-config here: if args.zookeeper is None: code, out = commands.getstatusoutput(args.binary + "-extract-from-config --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') @@ -147,12 +152,12 @@ def main(args): break case_file = os.path.join(suite_dir, case) - if os.path.isfile(case_file) and (case.endswith('.sh') or case.endswith('.py') or case.endswith('.sql')): - (name, ext) = os.path.splitext(case) + (name, ext) = os.path.splitext(case) + + if os.path.isfile(case_file) and (ext == '.sql' or ext == '.sh' or ext == '.py'): report_testcase = et.Element("testcase", attrib = {"name": name}) try: - print "{0:70}".format(name + ": "), sys.stdout.flush() @@ -190,9 +195,9 @@ def main(args): stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr' if ext == '.sql': - command = "{0} --multiquery < {1} > {2} 2> {3}".format(args.client, case_file, stdout_file, stderr_file) + command = "{} --server_logs_level={} --multiquery < {} > {} 2> {}".format(args.client, server_logs_level, case_file, stdout_file, stderr_file) else: - command = "{0} > {1} 2> {2}".format(case_file, stdout_file, stderr_file) + command = "{} > {} 2> {}".format(case_file, stdout_file, stderr_file) proc = Popen(command, shell = True) start_time = datetime.now() @@ -364,7 +369,5 @@ if __name__ == '__main__': args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST") if os.getenv("CLICKHOUSE_PORT_TCP"): args.client += ' --port=' + os.getenv("CLICKHOUSE_PORT_TCP") - # Will print all warnings and errors - args.client += ' --server_logs_level=warning' main(args) diff --git a/dbms/tests/queries/0_stateless/00534_long_functions_bad_arguments.lib b/dbms/tests/queries/0_stateless/00534_long_functions_bad_arguments.lib index 531b54a64c0..892581ab298 100755 --- a/dbms/tests/queries/0_stateless/00534_long_functions_bad_arguments.lib +++ b/dbms/tests/queries/0_stateless/00534_long_functions_bad_arguments.lib @@ -1,8 +1,5 @@ #!/usr/bin/env bash -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - function test_variant { perl -E "say \$_ for map {chomp; (qq{$1})} qx{$CLICKHOUSE_CLIENT -q 'SELECT name FROM system.functions ORDER BY name;'}" | $CLICKHOUSE_CLIENT -n --ignore-error >/dev/null 2>&1 $CLICKHOUSE_CLIENT -q "SELECT 'Still alive'" diff --git a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.reference b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.reference new file mode 100644 index 00000000000..9f61a33663e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.reference @@ -0,0 +1,5 @@ +1000 +0 +1 1 1 +0 +1 1 1 diff --git a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh new file mode 100755 index 00000000000..a6780ef7a5d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh @@ -0,0 +1,106 @@ +#!/usr/bin/env bash +set -e + +# Get all server logs +export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL="trace" + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +cur_name=${BASH_SOURCE[0]} +server_logs_file=$cur_name"_server.logs" +server_logs="--server_logs_file=$server_logs_file" +rm -f "$server_logs_file" + +settings="$server_logs --log_queries=1 --log_query_threads=1 --log_profile_events=1 --log_query_settings=1" + + +# Test insert logging on each block and checkPacket() method + +$CLICKHOUSE_CLIENT $settings -n -q " +DROP TABLE IF EXISTS test.null; +CREATE TABLE test.null (i UInt8) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();" + +head -c 1000 /dev/zero | $CLICKHOUSE_CLIENT $settings --max_insert_block_size=10 --min_insert_block_size_rows=1 --min_insert_block_size_bytes=1 -q "INSERT INTO test.null FORMAT RowBinary" + +$CLICKHOUSE_CLIENT $settings -n -q " +SELECT count() FROM test.null; +DROP TABLE test.null;" + +(( `cat "$server_logs_file" | wc -l` >= 110 )) || echo Fail + + +# Check ProfileEvents in query_log + +heavy_cpu_query="SELECT ignore(sum(sipHash64(hex(sipHash64(hex(sipHash64(hex(number)))))))) FROM (SELECT * FROM system.numbers_mt LIMIT 1000000)" +$CLICKHOUSE_CLIENT $settings --max_threads=1 -q "$heavy_cpu_query" +$CLICKHOUSE_CLIENT $settings -q "SYSTEM FLUSH SYSTEM TABLES" +$CLICKHOUSE_CLIENT $settings -q " +WITH + any(query_duration_ms*1000) AS duration, + sumIf(PV, PN = 'RealTimeMicroseconds') AS threads_realtime, + sumIf(PV, PN IN ('UserTimeMicroseconds', 'SystemTimeMicroseconds', 'OSIOWaitMicroseconds')) AS threads_time_user_system_io +SELECT + --duration, threads_realtime, threads_time_user_system_io, + threads_realtime >= 0.99 * duration, + threads_realtime >= threads_time_user_system_io, + any(length(thread_numbers)) >= 1 + FROM + (SELECT * FROM system.query_log WHERE event_date >= today()-1 AND type=2 AND query='$heavy_cpu_query' ORDER BY event_time DESC LIMIT 1) + ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV" + + +# Check ProfileEvents in query_thread_log + +$CLICKHOUSE_CLIENT $settings --max_threads=3 -q "$heavy_cpu_query" +$CLICKHOUSE_CLIENT $settings -q "SYSTEM FLUSH SYSTEM TABLES" +query_id=`$CLICKHOUSE_CLIENT $settings -q "SELECT query_id FROM system.query_log WHERE event_date >= today()-1 AND type=2 AND query='$heavy_cpu_query' ORDER BY event_time DESC LIMIT 1"` +query_elapsed=`$CLICKHOUSE_CLIENT $settings -q "SELECT query_duration_ms*1000 FROM system.query_log WHERE event_date >= today()-1 AND type=2 AND query_id='$query_id' ORDER BY event_time DESC LIMIT 1"` +threads=`$CLICKHOUSE_CLIENT $settings -q "SELECT length(thread_numbers) FROM system.query_log WHERE event_date >= today()-1 AND type=2 AND query_id='$query_id' ORDER BY event_time DESC LIMIT 1"` + +$CLICKHOUSE_CLIENT $settings -q " +SELECT + -- max(thread_realtime), $query_elapsed, max(thread_time_user_system_io), + 0.9 * $query_elapsed <= max(thread_realtime) AND max(thread_realtime) <= 1.1 * $query_elapsed, + 0.7 * $query_elapsed <= max(thread_time_user_system_io) AND max(thread_time_user_system_io) <= 1.3 * $query_elapsed, + uniqExact(thread_number) = $threads +FROM +( + SELECT + thread_number, + sumIf(PV, PN = 'RealTimeMicroseconds') AS thread_realtime, + sumIf(PV, PN IN ('UserTimeMicroseconds', 'SystemTimeMicroseconds', 'OSIOWaitMicroseconds')) AS thread_time_user_system_io + FROM + (SELECT * FROM system.query_thread_log WHERE event_date >= today()-1 AND query_id='$query_id') + ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV + GROUP BY thread_number +) +" + +# Check per-thread and per-query ProfileEvents consistency + +$CLICKHOUSE_CLIENT $settings -q " +SELECT PN, PVq, PVt FROM +( + SELECT PN, sum(PV) AS PVt + FROM system.query_thread_log + ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV + WHERE event_date >= today()-1 AND query_id='$query_id' + GROUP BY PN +) +ANY INNER JOIN +( + SELECT PN, PV AS PVq + FROM system.query_log + ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV + WHERE event_date >= today()-1 AND query_id='$query_id' +) +USING PN +WHERE + NOT PN IN ('ContextLock') AND + NOT (PVq <= PVt AND PVt <= 1.1 * PVq) +" + + +# Clean +#rm "$server_logs_file" diff --git a/dbms/tests/queries/shell_config.sh b/dbms/tests/queries/shell_config.sh index 00b7b67c50a..63bb323fa1a 100644 --- a/dbms/tests/queries/shell_config.sh +++ b/dbms/tests/queries/shell_config.sh @@ -1,6 +1,8 @@ export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"} -export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="${CLICKHOUSE_BINARY}-client --server_logs_level=warning"} +export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="${CLICKHOUSE_BINARY}-client"} +export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL:="warning"} +export CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --server_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}" export CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"} export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:="/etc/clickhouse-server/config.xml"} From e5e73d4a9b724f72e9ea3afedc1851b0afd2c628 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Thu, 14 Jun 2018 17:29:42 +0300 Subject: [PATCH 082/315] Add CAP_NET_ADMIN capability check. [#CLICKHOUSE-2910] --- dbms/programs/server/Server.cpp | 8 ++ dbms/src/Client/Connection.cpp | 4 +- dbms/src/Client/Connection.h | 2 +- dbms/src/Client/MultiplexedConnections.cpp | 2 +- dbms/src/Common/BackgroundSchedulePool.cpp | 1 - dbms/src/Common/CurrentThread.cpp | 4 +- dbms/src/Common/TaskStatsInfoGetter.cpp | 122 ++++++++++++--------- dbms/src/Common/TaskStatsInfoGetter.h | 26 +++-- dbms/src/Common/ThreadStatus.cpp | 7 +- dbms/src/Common/ThreadStatus.h | 1 + 10 files changed, 112 insertions(+), 65 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 368a8cdfa18..9317d90a564 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -336,6 +337,13 @@ int Server::main(const std::vector & /*args*/) dns_cache_updater = std::make_unique(*global_context); } + if (!TaskStatsInfoGetter::checkProcessHasRequiredPermissions()) + { + LOG_INFO(log, "It looks like the process has not CAP_NET_ADMIN capability, some performance statistics will be disabled." + " It could happen due to incorrect clickhouse package installation." + " You could resolve the problem manually calling 'sudo setcap cap_net_admin=+ep /usr/bin/clickhouse'"); + } + { Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index dd32c702dfc..959821a6bb5 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -505,7 +505,7 @@ bool Connection::poll(size_t timeout_microseconds) } -bool Connection::hasReadBufferPendingData() const +bool Connection::hasReadPendingData() const { return last_input_packet_type.has_value() || static_cast(*in).hasPendingData(); } @@ -516,7 +516,7 @@ std::optional Connection::checkPacket(size_t timeout_microseconds) if (last_input_packet_type.has_value()) return last_input_packet_type; - if (hasReadBufferPendingData() || poll(timeout_microseconds)) + if (hasReadPendingData() || poll(timeout_microseconds)) { // LOG_TRACE(log_wrapper.get(), "Receiving packet type"); UInt64 packet_type; diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index f76b8edba25..ef10fa42e68 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -139,7 +139,7 @@ public: bool poll(size_t timeout_microseconds = 0); /// Check, if has data in read buffer. - bool hasReadBufferPendingData() const; + bool hasReadPendingData() const; /// Checks if there is input data in connection and reads packet ID. std::optional checkPacket(size_t timeout_microseconds = 0); diff --git a/dbms/src/Client/MultiplexedConnections.cpp b/dbms/src/Client/MultiplexedConnections.cpp index 6c534ff53f4..3e88a20caa3 100644 --- a/dbms/src/Client/MultiplexedConnections.cpp +++ b/dbms/src/Client/MultiplexedConnections.cpp @@ -277,7 +277,7 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead for (const ReplicaState & state : replica_states) { Connection * connection = state.connection; - if ((connection != nullptr) && connection->hasReadBufferPendingData()) + if ((connection != nullptr) && connection->hasReadPendingData()) read_list.push_back(*connection->socket); } diff --git a/dbms/src/Common/BackgroundSchedulePool.cpp b/dbms/src/Common/BackgroundSchedulePool.cpp index 6c15e374b46..594dffcedd1 100644 --- a/dbms/src/Common/BackgroundSchedulePool.cpp +++ b/dbms/src/Common/BackgroundSchedulePool.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 5abc4888275..183d3c284b5 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -17,11 +17,13 @@ namespace ErrorCodes static ThreadStatusPtr getCurrentThreadImpl() { +#ifndef NDEBUG if (!current_thread || current_thread.use_count() <= 0) throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR); if (Poco::ThreadNumber::get() != current_thread->thread_number) throw Exception("Current thread has different thread number", ErrorCodes::LOGICAL_ERROR); +#endif return current_thread; } @@ -144,7 +146,7 @@ void CurrentThread::attachSystemLogsQueue(const std::shared_ptr std::shared_ptr CurrentThread::getSystemLogsQueue() { /// NOTE: this method could be called at early server startup stage - /// NOTE: this method could be called in ThreadStatus destructor, therefore we make use_count() check + /// NOTE: this method could be called in ThreadStatus destructor, therefore we make use_count() check just in case if (!current_thread || current_thread.use_count() <= 0) return nullptr; diff --git a/dbms/src/Common/TaskStatsInfoGetter.cpp b/dbms/src/Common/TaskStatsInfoGetter.cpp index 408ca8c0a28..04715f7d1ca 100644 --- a/dbms/src/Common/TaskStatsInfoGetter.cpp +++ b/dbms/src/Common/TaskStatsInfoGetter.cpp @@ -105,7 +105,7 @@ int send_cmd(int sock_fd, __u16 nlmsg_type, __u32 nlmsg_pid, } -int get_family_id(int nl_sock_fd) noexcept +UInt16 get_family_id(int nl_sock_fd) noexcept { struct { @@ -121,7 +121,7 @@ int get_family_id(int nl_sock_fd) noexcept strlen(TASKSTATS_GENL_NAME) + 1)) return 0; - int id = 0; + UInt16 id = 0; ssize_t rep_len = ::recv(nl_sock_fd, &ans, sizeof(ans), 0); if (ans.n.nlmsg_type == NLMSG_ERROR || (rep_len < 0) || !NLMSG_OK((&ans.n), rep_len)) return 0; @@ -135,32 +135,54 @@ int get_family_id(int nl_sock_fd) noexcept return id; } -bool get_taskstats(int nl_sock_fd, int nl_family_id, pid_t xxxid, ::taskstats & out_stats, Exception * out_exception = nullptr) +#pragma GCC diagnostic pop +} + + +TaskStatsInfoGetter::TaskStatsInfoGetter() = default; + +void TaskStatsInfoGetter::init() { - if (send_cmd(nl_sock_fd, nl_family_id, xxxid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &xxxid, sizeof(pid_t))) + if (netlink_socket_fd >= 0) + return; + + netlink_socket_fd = ::socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC); + if (netlink_socket_fd < 0) + throwFromErrno("Can't create PF_NETLINK socket"); + + ::sockaddr_nl addr; + memset(&addr, 0, sizeof(addr)); + addr.nl_family = AF_NETLINK; + + if (::bind(netlink_socket_fd, reinterpret_cast<::sockaddr *>(&addr), sizeof(addr)) < 0) + throwFromErrno("Can't bind PF_NETLINK socket"); + + netlink_family_id = get_family_id(netlink_socket_fd); +} + +bool TaskStatsInfoGetter::getStatImpl(int tid, ::taskstats & out_stats, bool throw_on_error) +{ + init(); + + if (send_cmd(netlink_socket_fd, netlink_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(pid_t))) throwFromErrno("Can't send a Netlink command"); NetlinkMessage msg; - int rv = ::recv(nl_sock_fd, &msg, sizeof(msg), 0); + ssize_t rv = ::recv(netlink_socket_fd, &msg, sizeof(msg), 0); if (msg.n.nlmsg_type == NLMSG_ERROR || !NLMSG_OK((&msg.n), rv)) { ::nlmsgerr * err = static_cast<::nlmsgerr *>(NLMSG_DATA(&msg)); - Exception e("Can't get Netlink response, error=" + std::to_string(err->error), ErrorCodes::NETLINK_ERROR); - - if (out_exception) - { - *out_exception = std::move(e); + if (throw_on_error) + throw Exception("Can't get Netlink response, error=" + std::to_string(err->error), ErrorCodes::NETLINK_ERROR); + else return false; - } - - throw Exception(std::move(e)); } rv = GENLMSG_PAYLOAD(&msg.n); ::nlattr * na = static_cast<::nlattr *>(GENLMSG_DATA(&msg)); - int len = 0; + ssize_t len = 0; while (len < rv) { @@ -191,49 +213,21 @@ bool get_taskstats(int nl_sock_fd, int nl_family_id, pid_t xxxid, ::taskstats & return true; } -#pragma GCC diagnostic pop - +void TaskStatsInfoGetter::getStat(::taskstats & stat, int tid) +{ + tid = tid < 0 ? getDefaultTid() : tid; + getStatImpl(tid, stat, true); } - -TaskStatsInfoGetter::TaskStatsInfoGetter() +bool TaskStatsInfoGetter::tryGetStat(::taskstats & stat, int tid) { - netlink_socket_fd = socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC); - if (netlink_socket_fd < 0) - throwFromErrno("Can't create PF_NETLINK socket"); - - ::sockaddr_nl addr; - memset(&addr, 0, sizeof(addr)); - addr.nl_family = AF_NETLINK; - - if (::bind(netlink_socket_fd, reinterpret_cast<::sockaddr *>(&addr), sizeof(addr)) < 0) - throwFromErrno("Can't bind PF_NETLINK socket"); - - netlink_family_id = get_family_id(netlink_socket_fd); - - initial_tid = getCurrentTID(); -} - -void TaskStatsInfoGetter::getStat(::taskstats & stat, int tid) const -{ - if (tid < 0) - tid = initial_tid; - - get_taskstats(netlink_socket_fd, netlink_family_id, tid, stat); -} - -bool TaskStatsInfoGetter::tryGetStat(::taskstats & stat, int tid) const -{ - if (tid < 0) - tid = initial_tid; - - Exception e; - return get_taskstats(netlink_socket_fd, netlink_family_id, tid, stat, &e); + tid = tid < 0 ? getDefaultTid() : tid; + return getStatImpl(tid, stat, false); } TaskStatsInfoGetter::~TaskStatsInfoGetter() { - if (netlink_socket_fd > -1) + if (netlink_socket_fd >= 0) close(netlink_socket_fd); } @@ -242,5 +236,33 @@ int TaskStatsInfoGetter::getCurrentTID() return static_cast(syscall(SYS_gettid)); } +int TaskStatsInfoGetter::getDefaultTid() +{ + if (default_tid < 0) + default_tid = getCurrentTID(); + + return default_tid; +} + +bool TaskStatsInfoGetter::checkProcessHasRequiredPermissions() +{ + /// 0 - wasn't checked + /// 1 - checked, has no permissions + /// 2 - checked, has permissions + static std::atomic premissions_check_status{0}; + + int status = premissions_check_status.load(std::memory_order_relaxed); + + if (status == 0) + { + TaskStatsInfoGetter getter; + ::taskstats stat; + + status = getter.tryGetStat(stat) ? 2 : 1; + premissions_check_status.store(status, std::memory_order_relaxed); + } + + return status == 2; +} } diff --git a/dbms/src/Common/TaskStatsInfoGetter.h b/dbms/src/Common/TaskStatsInfoGetter.h index 3add5055ece..7b0decd7571 100644 --- a/dbms/src/Common/TaskStatsInfoGetter.h +++ b/dbms/src/Common/TaskStatsInfoGetter.h @@ -7,6 +7,9 @@ struct taskstats; namespace DB { +class Exception; + + /// Get taskstat infor from OS kernel via Netlink protocol class TaskStatsInfoGetter { @@ -15,19 +18,28 @@ public: TaskStatsInfoGetter(); TaskStatsInfoGetter(const TaskStatsInfoGetter &) = delete; - void getStat(::taskstats & stat, int tid = -1) const; - bool tryGetStat(::taskstats & stat, int tid = -1) const; - - /// Returns Linux internal thread id - static int getCurrentTID(); + void getStat(::taskstats & stat, int tid = -1); + bool tryGetStat(::taskstats & stat, int tid = -1); ~TaskStatsInfoGetter(); + /// Make a syscall and returns Linux thread id + static int getCurrentTID(); + + /// Whether the current process has permissions (sudo or cap_net_admin capabilties) to get taskstats info + static bool checkProcessHasRequiredPermissions(); + private: + /// Caches current thread tid to avoid extra sys calls + int getDefaultTid(); + int default_tid = -1; + + bool getStatImpl(int tid, ::taskstats & out_stats, bool throw_on_error = false); + void init(); + int netlink_socket_fd = -1; - int netlink_family_id = 0; - int initial_tid = -1; + UInt16 netlink_family_id = 0; }; } diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 37d464f3a04..e8e716c5363 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -287,7 +287,9 @@ void ThreadStatus::attachQuery( ++queries_started; *last_rusage = RusageCounters::current(query_start_time_nanoseconds); - *last_taskstats = TasksStatsCounters::current(); + has_permissions_for_taskstats = TaskStatsInfoGetter::checkProcessHasRequiredPermissions(); + if (has_permissions_for_taskstats) + *last_taskstats = TasksStatsCounters::current(); } void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) @@ -335,7 +337,8 @@ void ThreadStatus::updatePerfomanceCountersImpl() try { RusageCounters::updateProfileEvents(*last_rusage, performance_counters); - TasksStatsCounters::updateProfileEvents(*last_taskstats, performance_counters); + if (has_permissions_for_taskstats) + TasksStatsCounters::updateProfileEvents(*last_taskstats, performance_counters); } catch (...) { diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 8d846202e04..5ba54d303c9 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -135,6 +135,7 @@ protected: std::unique_ptr last_rusage; std::unique_ptr last_taskstats; std::unique_ptr taskstats_getter; + bool has_permissions_for_taskstats = false; public: class CurrentThreadScope; From 608895d9ff2cdcae54a0b209e423301e40b3aaaa Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Thu, 14 Jun 2018 18:33:59 +0300 Subject: [PATCH 083/315] Disable compression for Log packets. [#CLICKHOUSE-2910] --- dbms/programs/server/TCPHandler.cpp | 28 +++++++------------ dbms/programs/server/TCPHandler.h | 1 - dbms/src/Client/Connection.cpp | 20 ++++++------- dbms/src/Core/Defines.h | 2 +- .../00623_truncate_table_throw_exception.sh | 2 +- 5 files changed, 22 insertions(+), 31 deletions(-) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 8d635181dae..d2af5c25f0f 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -742,7 +742,14 @@ void TCPHandler::initBlockOutput(const Block & block) { if (!state.block_out) { - initOutputBuffers(); + if (!state.maybe_compressed_out) + { + if (state.compression == Protocol::Compression::Enable) + state.maybe_compressed_out = std::make_shared( + *out, CompressionSettings(query_context.getSettingsRef())); + else + state.maybe_compressed_out = out; + } state.block_out = std::make_shared( *state.maybe_compressed_out, @@ -755,29 +762,15 @@ void TCPHandler::initLogsBlockOutput(const Block & block) { if (!state.logs_block_out) { - initOutputBuffers(); - + /// Use uncompressed stream since log blocks usually contain only one row state.logs_block_out = std::make_shared( - *state.maybe_compressed_out, + *out, client_revision, block.cloneEmpty()); } } -void TCPHandler::initOutputBuffers() -{ - if (!state.maybe_compressed_out) - { - if (state.compression == Protocol::Compression::Enable) - state.maybe_compressed_out = std::make_shared( - *out, CompressionSettings(query_context.getSettingsRef())); - else - state.maybe_compressed_out = out; - } -} - - bool TCPHandler::isQueryCancelled() { if (state.is_cancelled || state.sent_all_data) @@ -835,7 +828,6 @@ void TCPHandler::sendLogData(const Block & block) writeStringBinary("", *out); state.logs_block_out->write(block); - state.maybe_compressed_out->next(); out->next(); } diff --git a/dbms/programs/server/TCPHandler.h b/dbms/programs/server/TCPHandler.h index df64d6ef89b..bc8a484f48f 100644 --- a/dbms/programs/server/TCPHandler.h +++ b/dbms/programs/server/TCPHandler.h @@ -157,7 +157,6 @@ private: void initBlockInput(); void initBlockOutput(const Block & block); void initLogsBlockOutput(const Block & block); - void initOutputBuffers(); bool isQueryCancelled(); diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 959821a6bb5..733561556b9 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -639,13 +639,7 @@ Block Connection::receiveDataImpl(BlockInputStreamPtr & stream) void Connection::initInputBuffers() { - if (!maybe_compressed_in) - { - if (compression == Protocol::Compression::Enable) - maybe_compressed_in = std::make_shared(*in); - else - maybe_compressed_in = in; - } + } @@ -653,7 +647,14 @@ void Connection::initBlockInput() { if (!block_in) { - initInputBuffers(); + if (!maybe_compressed_in) + { + if (compression == Protocol::Compression::Enable) + maybe_compressed_in = std::make_shared(*in); + else + maybe_compressed_in = in; + } + block_in = std::make_shared(*maybe_compressed_in, server_revision); } } @@ -663,9 +664,8 @@ void Connection::initBlockLogsInput() { if (!block_logs_in) { - initInputBuffers(); /// Have to return superset of SystemLogsQueue::getSampleBlock() columns - block_logs_in = std::make_shared(*maybe_compressed_in, server_revision); + block_logs_in = std::make_shared(*in, server_revision); } } diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 16632ccefc3..877ad09f839 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -61,7 +61,7 @@ #define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226 #define DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE 54337 #define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372 -#define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54382 +#define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54386 /// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change. #define DBMS_TCP_PROTOCOL_VERSION 54226 diff --git a/dbms/tests/queries/0_stateless/00623_truncate_table_throw_exception.sh b/dbms/tests/queries/0_stateless/00623_truncate_table_throw_exception.sh index 3b6d7581b17..7f9ae704b98 100755 --- a/dbms/tests/queries/0_stateless/00623_truncate_table_throw_exception.sh +++ b/dbms/tests/queries/0_stateless/00623_truncate_table_throw_exception.sh @@ -15,7 +15,7 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO test_truncate.test_view_depend VALUES( ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_truncate.test_view;" ${CLICKHOUSE_CLIENT} --query "SELECT '========Execute Truncate========';" -echo `${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE test_truncate.test_view;" 2>&1 | grep -c "Code: 48.*Truncate is not supported by storage View"` +echo `${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE test_truncate.test_view;" --server_logs_file=/dev/null 2>&1 | grep -c "Code: 48.*Truncate is not supported by storage View"` ${CLICKHOUSE_CLIENT} --query "SELECT '========After Truncate========';" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_truncate.test_view;" From a0e9d17bfdfa4a65f14de2a9c885a438c1c43b92 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Fri, 15 Jun 2018 16:45:19 +0300 Subject: [PATCH 084/315] Add query_id and host_name to server logs. [#CLICKHOUSE-2910] --- dbms/programs/client/Client.cpp | 2 +- dbms/src/Common/CurrentThread.cpp | 10 ++- dbms/src/Common/CurrentThread.h | 5 ++ dbms/src/Common/DNSResolver.cpp | 21 +++++ dbms/src/Common/DNSResolver.h | 3 + dbms/src/Common/ThreadStatus.cpp | 4 +- dbms/src/Common/ThreadStatus.h | 2 +- dbms/src/Core/SystemLogsQueue.cpp | 4 +- dbms/src/Core/SystemLogsQueue.h | 2 +- .../AsynchronousBlockInputStream.cpp | 3 +- .../IProfilingBlockInputStream.cpp | 2 +- .../DataStreams/SystemLogsRowOutputStream.cpp | 82 ++++++++++++------- .../DataStreams/SystemLogsRowOutputStream.h | 4 - dbms/src/Functions/FunctionsMiscellaneous.cpp | 3 +- .../src/Interpreters/ClickHouseLogChannel.cpp | 3 + .../System/StorageSystemProcesses.cpp | 45 ++++------ .../Storages/System/StorageSystemProcesses.h | 2 +- .../System/VirtualColumnsProcessor.cpp | 4 +- 18 files changed, 124 insertions(+), 77 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 92aeb91dcae..085a17bb412 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1225,7 +1225,7 @@ private: } } - logs_out_stream = SystemLogsRowOutputStream::create(*wb); + logs_out_stream = std::make_shared(*wb); logs_out_stream->writePrefix(); } } diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 183d3c284b5..6e792f29e8c 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -60,7 +60,7 @@ void CurrentThread::attachQueryFromSiblingThreadIfDetached(const ThreadStatusPtr void CurrentThread::updatePerformanceCounters() { - getCurrentThreadImpl()->updatePerfomanceCountersImpl(); + getCurrentThreadImpl()->updatePerformanceCountersImpl(); } ThreadStatusPtr CurrentThread::get() @@ -157,4 +157,12 @@ std::shared_ptr CurrentThread::getSystemLogsQueue() return current_thread->getSystemLogsQueue(); } +std::string CurrentThread::getCurrentQueryID() +{ + if (!current_thread || current_thread.use_count() <= 0 || !current_thread->parent_query) + return {}; + + return current_thread->parent_query->client_info.current_query_id; +} + } diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index 5ddd2b0fcee..e11870f47be 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace ProfileEvents @@ -47,6 +48,10 @@ public: static ProfileEvents::Counters & getProfileEvents(); static MemoryTracker & getMemoryTracker(); + + /// Returns a non-empty string if the thread is attached to a query + static std::string getCurrentQueryID(); + static void updateProgressIn(const Progress & value); static void updateProgressOut(const Progress & value); diff --git a/dbms/src/Common/DNSResolver.cpp b/dbms/src/Common/DNSResolver.cpp index e3d9442deea..c82f302d7b2 100644 --- a/dbms/src/Common/DNSResolver.cpp +++ b/dbms/src/Common/DNSResolver.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -79,6 +80,10 @@ struct DNSResolver::Impl { SimpleCache cache_host; + /// Cached server host name + std::mutex mutex; + std::optional host_name; + /// If disabled, will not make cache lookups, will resolve addresses manually on each call std::atomic disable_cache{false}; }; @@ -108,6 +113,9 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, U void DNSResolver::dropCache() { impl->cache_host.drop(); + + std::unique_lock lock(impl->mutex); + impl->host_name.reset(); } void DNSResolver::setDisableCacheFlag(bool is_disabled) @@ -115,6 +123,19 @@ void DNSResolver::setDisableCacheFlag(bool is_disabled) impl->disable_cache = is_disabled; } +String DNSResolver::getHostName() +{ + if (impl->disable_cache) + return Poco::Net::DNS::hostName(); + + std::unique_lock lock(impl->mutex); + + if (!impl->host_name.has_value()) + impl->host_name.emplace(Poco::Net::DNS::hostName()); + + return *impl->host_name; +} + DNSResolver::~DNSResolver() = default; diff --git a/dbms/src/Common/DNSResolver.h b/dbms/src/Common/DNSResolver.h index fb3892e101f..097e646fa65 100644 --- a/dbms/src/Common/DNSResolver.h +++ b/dbms/src/Common/DNSResolver.h @@ -25,6 +25,9 @@ public: Poco::Net::SocketAddress resolveAddress(const std::string & host, UInt16 port); + /// Get this server host name + String getHostName(); + /// Disables caching void setDisableCacheFlag(bool is_disabled = true); diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index e8e716c5363..7614ee10c6d 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -303,7 +303,7 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) if (thread_state != ThreadState::AttachedToQuery && thread_state != ThreadState::QueryInitializing) throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); - updatePerfomanceCountersImpl(); + updatePerformanceCountersImpl(); try { @@ -332,7 +332,7 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) } -void ThreadStatus::updatePerfomanceCountersImpl() +void ThreadStatus::updatePerformanceCountersImpl() { try { diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 5ba54d303c9..a59ccff9127 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -104,7 +104,7 @@ protected: void logToQueryThreadLog(QueryThreadLog & thread_log); - void updatePerfomanceCountersImpl(); + void updatePerformanceCountersImpl(); std::atomic thread_state{ThreadState::DetachedFromQuery}; diff --git a/dbms/src/Core/SystemLogsQueue.cpp b/dbms/src/Core/SystemLogsQueue.cpp index aa7e336dc52..54b02be7fc5 100644 --- a/dbms/src/Core/SystemLogsQueue.cpp +++ b/dbms/src/Core/SystemLogsQueue.cpp @@ -20,6 +20,8 @@ Block SystemLogsQueue::getSampleBlock() return Block { {std::make_shared(), "event_time"}, {std::make_shared(), "event_time_microseconds"}, + {std::make_shared(), "host_name"}, + {std::make_shared(), "query_id"}, {std::make_shared(), "thread_number"}, {std::make_shared(), "priority"}, {std::make_shared(), "source"}, @@ -33,7 +35,7 @@ MutableColumns SystemLogsQueue::getSampleColumns() return sample_block.cloneEmptyColumns(); } -const char * SystemLogsQueue::getProrityName(int priority) +const char * SystemLogsQueue::getPriorityName(int priority) { /// See Poco::Message::Priority diff --git a/dbms/src/Core/SystemLogsQueue.h b/dbms/src/Core/SystemLogsQueue.h index f6b8d1b8d69..10535f245f9 100644 --- a/dbms/src/Core/SystemLogsQueue.h +++ b/dbms/src/Core/SystemLogsQueue.h @@ -18,7 +18,7 @@ public: static MutableColumns getSampleColumns(); /// Converts priority from Poco::Message::Priority to a string - static const char * getProrityName(int priority); + static const char * getPriorityName(int priority); }; using SystemLogsQueuePtr = std::shared_ptr; diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp index 25055001ecc..dda2378ef88 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp @@ -35,7 +35,8 @@ void AsynchronousBlockInputStream::next() { ready.reset(); - pool.schedule([this, main_thread=CurrentThread::get()] () { + pool.schedule([this, main_thread=CurrentThread::get()] () + { CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; try diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 4855ae88ddd..61ea4cfdf0b 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -293,7 +293,7 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) size_t total_rows = progress.total_rows; - constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds + constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds UInt64 total_elapsed_microseconds = info.total_stopwatch.elapsedMicroseconds(); if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) diff --git a/dbms/src/DataStreams/SystemLogsRowOutputStream.cpp b/dbms/src/DataStreams/SystemLogsRowOutputStream.cpp index 871589ae849..10ca4c30235 100644 --- a/dbms/src/DataStreams/SystemLogsRowOutputStream.cpp +++ b/dbms/src/DataStreams/SystemLogsRowOutputStream.cpp @@ -16,47 +16,67 @@ Block SystemLogsRowOutputStream::getHeader() const return SystemLogsQueue::getSampleBlock(); } - void SystemLogsRowOutputStream::write(const Block & block) { - for (size_t i = 0; i < block.rows(); ++i) - write(block, i); -} + auto & array_event_time = typeid_cast(*block.getByName("event_time").column).getData(); + auto & array_microseconds = typeid_cast(*block.getByName("event_time_microseconds").column).getData(); + auto & column_host_name = typeid_cast(*block.getByName("host_name").column); + auto & column_query_id = typeid_cast(*block.getByName("query_id").column); -void SystemLogsRowOutputStream::write(const Block & block, size_t row_num) -{ - UInt32 event_time = typeid_cast(*block.getByName("event_time").column).getData()[row_num]; - writeDateTimeText<'.', ':'>(event_time, wb); + auto & array_thread_number = typeid_cast(*block.getByName("thread_number").column).getData(); + auto & array_priority = typeid_cast(*block.getByName("priority").column).getData(); + auto & column_source = typeid_cast(*block.getByName("source").column); + auto & column_text = typeid_cast(*block.getByName("text").column); - UInt32 microseconds = typeid_cast(*block.getByName("event_time_microseconds").column).getData()[row_num]; - writeChar('.', wb); - writeChar('0' + ((microseconds / 100000) % 10), wb); - writeChar('0' + ((microseconds / 10000) % 10), wb); - writeChar('0' + ((microseconds / 1000) % 10), wb); - writeChar('0' + ((microseconds / 100) % 10), wb); - writeChar('0' + ((microseconds / 10) % 10), wb); - writeChar('0' + ((microseconds / 1) % 10), wb); + for (size_t row_num = 0; row_num < block.rows(); ++row_num) + { + auto event_time = array_event_time[row_num]; + writeDateTimeText<'.', ':'>(event_time, wb); - writeCString(" [ ", wb); - UInt32 thread_number = typeid_cast(*block.getByName("thread_number").column).getData()[row_num]; - writeIntText(thread_number, wb); - writeCString(" ] <", wb); + auto microseconds = array_microseconds[row_num]; + writeChar('.', wb); + writeChar('0' + ((microseconds / 100000) % 10), wb); + writeChar('0' + ((microseconds / 10000) % 10), wb); + writeChar('0' + ((microseconds / 1000) % 10), wb); + writeChar('0' + ((microseconds / 100) % 10), wb); + writeChar('0' + ((microseconds / 10) % 10), wb); + writeChar('0' + ((microseconds / 1) % 10), wb); - Int8 priority = typeid_cast(*block.getByName("priority").column).getData()[row_num]; - writeString(SystemLogsQueue::getProrityName(priority), wb); - writeCString("> ", wb); + auto host_name = column_host_name.getDataAt(row_num); + if (host_name.size) + { + writeCString(" [ ", wb); + writeString(host_name, wb); + writeCString(" ]", wb); + } - writeString(typeid_cast(*block.getByName("source").column).getDataAt(row_num), wb); - writeCString(": ", wb); - writeString(typeid_cast(*block.getByName("text").column).getDataAt(row_num), wb); + auto query_id = column_query_id.getDataAt(row_num); + if (query_id.size) + { + writeCString(" { ", wb); + writeString(query_id, wb); + writeCString(" }", wb); + } - writeChar('\n', wb); -} + UInt32 thread_number = array_thread_number[row_num]; + writeCString(" [ ", wb); + writeIntText(thread_number, wb); + writeCString(" ] <", wb); -BlockOutputStreamPtr SystemLogsRowOutputStream::create(WriteBuffer & buf_out) -{ - return std::make_shared(buf_out); + Int8 priority = array_priority[row_num]; + writeString(SystemLogsQueue::getPriorityName(priority), wb); + writeCString("> ", wb); + + auto source = column_source.getDataAt(row_num); + writeString(source, wb); + writeCString(": ", wb); + + auto text = column_text.getDataAt(row_num); + writeString(text, wb); + + writeChar('\n', wb); + } } } diff --git a/dbms/src/DataStreams/SystemLogsRowOutputStream.h b/dbms/src/DataStreams/SystemLogsRowOutputStream.h index d94e209ea49..217209da608 100644 --- a/dbms/src/DataStreams/SystemLogsRowOutputStream.h +++ b/dbms/src/DataStreams/SystemLogsRowOutputStream.h @@ -13,14 +13,10 @@ class SystemLogsRowOutputStream : public IBlockOutputStream { public: - static BlockOutputStreamPtr create(WriteBuffer & buf_out); - SystemLogsRowOutputStream(WriteBuffer & buf_out) : wb(buf_out) {} Block getHeader() const override; - void write(const Block & block, size_t row_num); - void write(const Block & block) override; void flush() override diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index 65b7c970a6a..33f6a5801dd 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -33,6 +33,7 @@ #include #include #include +#include namespace DB @@ -156,7 +157,7 @@ public: void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override { block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst( - input_rows_count, Poco::Net::DNS::hostName())->convertToFullColumnIfConst(); + input_rows_count, DNSResolver::instance().getHostName())->convertToFullColumnIfConst(); } }; diff --git a/dbms/src/Interpreters/ClickHouseLogChannel.cpp b/dbms/src/Interpreters/ClickHouseLogChannel.cpp index ea69d2dc088..be7dc891032 100644 --- a/dbms/src/Interpreters/ClickHouseLogChannel.cpp +++ b/dbms/src/Interpreters/ClickHouseLogChannel.cpp @@ -1,6 +1,7 @@ #include "ClickHouseLogChannel.h" #include +#include #include #include @@ -33,6 +34,8 @@ void ClickHouseLogChannel::log(const Poco::Message & msg) size_t i = 0; columns[i++]->insert(static_cast(tv.tv_sec)); columns[i++]->insert(static_cast(tv.tv_usec)); + columns[i++]->insert(DNSResolver::instance().getHostName()); + columns[i++]->insert(CurrentThread::getCurrentQueryID()); columns[i++]->insert(static_cast(Poco::ThreadNumber::get())); columns[i++]->insert(static_cast(msg.getPriority())); columns[i++]->insert(msg.getSource()); diff --git a/dbms/src/Storages/System/StorageSystemProcesses.cpp b/dbms/src/Storages/System/StorageSystemProcesses.cpp index 48060ff6ee8..bcbc3f147c7 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.cpp +++ b/dbms/src/Storages/System/StorageSystemProcesses.cpp @@ -58,15 +58,13 @@ StorageSystemProcesses::StorageSystemProcesses(const std::string & name_) { "memory_usage", std::make_shared() }, { "peak_memory_usage", std::make_shared() }, { "query", std::make_shared() }, - })); - virtual_columns = ColumnsWithTypeAndName{ - { std::make_shared(std::make_shared()), "thread_numbers" }, - { std::make_shared(std::make_shared()), "ProfileEvents_Names" }, - { std::make_shared(std::make_shared()), "ProfileEvents_Values" }, - { std::make_shared(std::make_shared()), "Settings_Names" }, - { std::make_shared(std::make_shared()), "Settings_Values" } - }; + { "thread_numbers", std::make_shared(std::make_shared()) }, + { "ProfileEvents.Names", std::make_shared(std::make_shared()) }, + { "ProfileEvents.Values", std::make_shared(std::make_shared()) }, + { "Settings.Names", std::make_shared(std::make_shared()) }, + { "Settings.Values", std::make_shared(std::make_shared()) } + })); } @@ -79,20 +77,12 @@ BlockInputStreams StorageSystemProcesses::read( const unsigned /*num_streams*/) { processed_stage = QueryProcessingStage::FetchColumns; - - auto virtual_columns_processor = getVirtualColumnsProcessor(); - bool has_thread_numbers, has_profile_events_names, has_profile_events_values, has_settigns_names, has_settings_values; - std::vector flags{&has_thread_numbers, &has_profile_events_names, &has_profile_events_values, &has_settigns_names, &has_settings_values}; - - Names real_columns = virtual_columns_processor.process(column_names, flags); - check(real_columns); + check(column_names); Block res_block = getSampleBlock().cloneEmpty(); - virtual_columns_processor.appendVirtualColumns(res_block); MutableColumns res_columns = res_block.cloneEmptyColumns(); - ProcessList::Info info = context.getProcessList().getInfo(has_thread_numbers, has_profile_events_names || has_profile_events_values, - has_settigns_names || has_settings_values); + ProcessList::Info info = context.getProcessList().getInfo(true, true, true); for (const auto & process : info) { @@ -127,7 +117,6 @@ BlockInputStreams StorageSystemProcesses::read( res_columns[i++]->insert(process.peak_memory_usage); res_columns[i++]->insert(process.query); - if (has_thread_numbers) { Array threads_array; threads_array.reserve(process.thread_numbers.size()); @@ -136,24 +125,22 @@ BlockInputStreams StorageSystemProcesses::read( res_columns[i++]->insert(threads_array); } - if (has_profile_events_names || has_profile_events_values) { - IColumn * column_names = has_profile_events_names ? res_columns[i++].get() : nullptr; - IColumn * column_values = has_profile_events_values ? res_columns[i++].get() : nullptr; - process.profile_counters->dumpToArrayColumns(column_names, column_values, true); + IColumn * column_profile_events_names = res_columns[i++].get(); + IColumn * column_profile_events_values = res_columns[i++].get(); + process.profile_counters->dumpToArrayColumns(column_profile_events_names, column_profile_events_values, true); } - if (has_settigns_names || has_settings_values) { - IColumn * column_names = has_settigns_names ? res_columns[i++].get() : nullptr; - IColumn * column_values = has_settings_values ? res_columns[i++].get() : nullptr; + IColumn * column_settings_names = res_columns[i++].get(); + IColumn * column_settings_values = res_columns[i++].get(); if (process.query_settings) - process.query_settings->dumpToArrayColumns(column_names, column_values, true); + process.query_settings->dumpToArrayColumns(column_settings_names, column_settings_values, true); else { - column_names->insertDefault(); - column_values->insertDefault(); + column_settings_names->insertDefault(); + column_settings_values->insertDefault(); } } } diff --git a/dbms/src/Storages/System/StorageSystemProcesses.h b/dbms/src/Storages/System/StorageSystemProcesses.h index 59e7fa91d16..e2f7b35ae8b 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.h +++ b/dbms/src/Storages/System/StorageSystemProcesses.h @@ -13,7 +13,7 @@ class Context; /** Implements `processes` system table, which allows you to get information about the queries that are currently executing. */ -class StorageSystemProcesses : public ext::shared_ptr_helper, public StorageWithVirtualColumns +class StorageSystemProcesses : public ext::shared_ptr_helper, public IStorage { public: std::string getName() const override { return "SystemProcesses"; } diff --git a/dbms/src/Storages/System/VirtualColumnsProcessor.cpp b/dbms/src/Storages/System/VirtualColumnsProcessor.cpp index a126cc3f03a..1e41050c40d 100644 --- a/dbms/src/Storages/System/VirtualColumnsProcessor.cpp +++ b/dbms/src/Storages/System/VirtualColumnsProcessor.cpp @@ -3,7 +3,7 @@ namespace DB { -bool hasColumn(const ColumnsWithTypeAndName & columns, const String & column_name) +static bool hasColumn(const ColumnsWithTypeAndName & columns, const String & column_name) { for (const auto & column : columns) { @@ -15,7 +15,7 @@ bool hasColumn(const ColumnsWithTypeAndName & columns, const String & column_nam } -NameAndTypePair tryGetColumn(const ColumnsWithTypeAndName & columns, const String & column_name) +static NameAndTypePair tryGetColumn(const ColumnsWithTypeAndName & columns, const String & column_name) { for (const auto & column : columns) { From 392f6e0c6c3a0c0992aa97017c85301cd9f90516 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Fri, 15 Jun 2018 20:32:35 +0300 Subject: [PATCH 085/315] Print query_id in logs. Add own logging interface. [#CLICKHOUSE-2910] --- dbms/programs/client/Client.cpp | 4 +- dbms/programs/server/TCPHandler.cpp | 8 +-- dbms/programs/server/TCPHandler.h | 8 +-- dbms/src/Common/CurrentThread.cpp | 24 ++++--- dbms/src/Common/CurrentThread.h | 6 +- dbms/src/Common/ThreadStatus.cpp | 2 +- dbms/src/Common/ThreadStatus.h | 15 +++-- ...pp => InternalTextLogsRowOutputStream.cpp} | 32 +++++----- ...am.h => InternalTextLogsRowOutputStream.h} | 4 +- .../src/Interpreters/ClickHouseLogChannel.cpp | 49 --------------- dbms/src/Interpreters/ClickHouseLogChannel.h | 17 ----- .../InternalTextLogsQueue.cpp} | 10 +-- .../InternalTextLogsQueue.h} | 10 +-- ...4_performance_introspection_and_logging.sh | 2 +- libs/libdaemon/CMakeLists.txt | 6 ++ .../include/daemon/ExtendedLogChannel.h | 39 ++++++++++++ .../include/daemon/OwnFormattingChannel.h | 57 +++++++++++++++++ .../include/daemon/OwnPatternFormatter.h | 4 +- .../include/daemon/OwnSplitChannel.h | 35 +++++++++++ libs/libdaemon/src/BaseDaemon.cpp | 61 ++++++++---------- libs/libdaemon/src/ExtendedLogChannel.cpp | 27 ++++++++ libs/libdaemon/src/OwnFormattingChannel.cpp | 33 ++++++++++ libs/libdaemon/src/OwnPatternFormatter.cpp | 43 ++++++++----- libs/libdaemon/src/OwnSplitChannel.cpp | 62 +++++++++++++++++++ 24 files changed, 382 insertions(+), 176 deletions(-) rename dbms/src/DataStreams/{SystemLogsRowOutputStream.cpp => InternalTextLogsRowOutputStream.cpp} (83%) rename dbms/src/DataStreams/{SystemLogsRowOutputStream.h => InternalTextLogsRowOutputStream.h} (77%) delete mode 100644 dbms/src/Interpreters/ClickHouseLogChannel.cpp delete mode 100644 dbms/src/Interpreters/ClickHouseLogChannel.h rename dbms/src/{Core/SystemLogsQueue.cpp => Interpreters/InternalTextLogsQueue.cpp} (83%) rename dbms/src/{Core/SystemLogsQueue.h => Interpreters/InternalTextLogsQueue.h} (59%) create mode 100644 libs/libdaemon/include/daemon/ExtendedLogChannel.h create mode 100644 libs/libdaemon/include/daemon/OwnFormattingChannel.h create mode 100644 libs/libdaemon/include/daemon/OwnSplitChannel.h create mode 100644 libs/libdaemon/src/ExtendedLogChannel.cpp create mode 100644 libs/libdaemon/src/OwnFormattingChannel.cpp create mode 100644 libs/libdaemon/src/OwnSplitChannel.cpp diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 085a17bb412..3de914c909e 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -38,7 +38,7 @@ #include #include #include -#include +#include #include #include #include @@ -1225,7 +1225,7 @@ private: } } - logs_out_stream = std::make_shared(*wb); + logs_out_stream = std::make_shared(*wb); logs_out_stream->writePrefix(); } } diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index d2af5c25f0f..28983b2eea3 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -29,8 +29,8 @@ #include #include "TCPHandler.h" -#include -#include +#include +#include #include #include @@ -158,7 +158,7 @@ void TCPHandler::runImpl() if (client_revision >= DBMS_MIN_REVISION_WITH_SERVER_LOGS && query_context.getSettingsRef().server_logs_level.value != "none") { - state.logs_queue = std::make_shared(); + state.logs_queue = std::make_shared(); state.logs_queue->max_priority = Poco::Logger::parseLevel(query_context.getSettingsRef().server_logs_level.value); CurrentThread::attachSystemLogsQueue(state.logs_queue); } @@ -887,7 +887,7 @@ void TCPHandler::sendLogs() if (rows > 0) { - Block block = SystemLogsQueue::getSampleBlock(); + Block block = InternalTextLogsQueue::getSampleBlock(); block.setColumns(std::move(logs_columns)); sendLogData(block); } diff --git a/dbms/programs/server/TCPHandler.h b/dbms/programs/server/TCPHandler.h index bc8a484f48f..6e2ff7d5bee 100644 --- a/dbms/programs/server/TCPHandler.h +++ b/dbms/programs/server/TCPHandler.h @@ -5,13 +5,13 @@ #include #include #include -#include #include #include -#include -#include +#include #include #include +#include +#include #include #include "IServer.h" @@ -66,7 +66,7 @@ struct QueryState std::unique_ptr timeout_setter; /// A queue with internal logs that will be passed to client - SystemLogsQueuePtr logs_queue; + InternalTextLogsQueuePtr logs_queue; BlockOutputStreamPtr logs_block_out; void reset() diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 6e792f29e8c..7b07bd44a7e 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -39,11 +40,12 @@ void CurrentThread::attachQuery(QueryStatus * parent_process) ThreadStatusPtr thread = getCurrentThreadImpl(); if (!parent_process) - thread->attachQuery(nullptr, nullptr, nullptr, CurrentThread::getSystemLogsQueue()); + thread->attachQuery(nullptr, nullptr, nullptr, CurrentThread::getInternalTextLogsQueue()); else { thread->attachQuery( - parent_process, &parent_process->performance_counters, &parent_process->memory_tracker, CurrentThread::getSystemLogsQueue()); + parent_process, &parent_process->performance_counters, &parent_process->memory_tracker, + CurrentThread::getInternalTextLogsQueue()); } } @@ -114,7 +116,7 @@ void CurrentThread::attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thr QueryStatus * parent_query; ProfileEvents::Counters * parent_counters; MemoryTracker * parent_memory_tracker; - SystemLogsQueueWeakPtr logs_queue_ptr; + InternalTextLogsQueueWeakPtr logs_queue_ptr; { /// NOTE: It is almost the only place where ThreadStatus::mutex is required /// In other cases ThreadStatus must be accessed only from the current_thread @@ -138,12 +140,12 @@ void CurrentThread::attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thr thread->attachQuery(parent_query, parent_counters, parent_memory_tracker, logs_queue_ptr, check_detached); } -void CurrentThread::attachSystemLogsQueue(const std::shared_ptr & logs_queue) +void CurrentThread::attachSystemLogsQueue(const std::shared_ptr & logs_queue) { getCurrentThreadImpl()->attachSystemLogsQueue(logs_queue); } -std::shared_ptr CurrentThread::getSystemLogsQueue() +std::shared_ptr CurrentThread::getInternalTextLogsQueue() { /// NOTE: this method could be called at early server startup stage /// NOTE: this method could be called in ThreadStatus destructor, therefore we make use_count() check just in case @@ -154,15 +156,21 @@ std::shared_ptr CurrentThread::getSystemLogsQueue() if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died) return nullptr; - return current_thread->getSystemLogsQueue(); + return current_thread->getInternalTextLogsQueue(); } std::string CurrentThread::getCurrentQueryID() { - if (!current_thread || current_thread.use_count() <= 0 || !current_thread->parent_query) + if (!current_thread || current_thread.use_count() <= 0) return {}; - return current_thread->parent_query->client_info.current_query_id; + if (current_thread->parent_query) + return current_thread->parent_query->client_info.current_query_id; + + if (current_thread->query_context) + return current_thread->query_context->getClientInfo().current_query_id; + + return {}; } } diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index e11870f47be..7fc9a03a7e0 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -18,7 +18,7 @@ class QueryStatus; class ThreadStatus; struct Progress; using ThreadStatusPtr = std::shared_ptr; -class SystemLogsQueue; +class InternalTextLogsQueue; class CurrentThread @@ -32,8 +32,8 @@ public: static void initializeQuery(); /// A logs queue used by TCPHandler to pass logs to a client - static void attachSystemLogsQueue(const std::shared_ptr & logs_queue); - static std::shared_ptr getSystemLogsQueue(); + static void attachSystemLogsQueue(const std::shared_ptr & logs_queue); + static std::shared_ptr getInternalTextLogsQueue(); /// You must call one of these methods when create a child thread: /// Bundles the current thread with a query diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 7614ee10c6d..416425833fb 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -228,7 +228,7 @@ void ThreadStatus::attachQuery( QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker, - const SystemLogsQueueWeakPtr & logs_queue_ptr_, + const InternalTextLogsQueueWeakPtr & logs_queue_ptr_, bool check_detached) { if (thread_state == ThreadState::AttachedToQuery) diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index a59ccff9127..afe213eeb90 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -22,9 +22,9 @@ class QueryThreadLog; struct TasksStatsCounters; struct RusageCounters; class TaskStatsInfoGetter; -class SystemLogsQueue; -using SystemLogsQueuePtr = std::shared_ptr; -using SystemLogsQueueWeakPtr = std::weak_ptr; +class InternalTextLogsQueue; +using InternalTextLogsQueuePtr = std::shared_ptr; +using InternalTextLogsQueueWeakPtr = std::weak_ptr; using ThreadStatusPtr = std::shared_ptr; @@ -68,13 +68,12 @@ public: return thread_state.load(std::memory_order_relaxed); } - SystemLogsQueuePtr getSystemLogsQueue() const + InternalTextLogsQueuePtr getInternalTextLogsQueue() const { - std::lock_guard lock(mutex); return thread_state == Died ? nullptr : logs_queue_ptr.lock(); } - void attachSystemLogsQueue(const SystemLogsQueuePtr & logs_queue) + void attachSystemLogsQueue(const InternalTextLogsQueuePtr & logs_queue) { std::lock_guard lock(mutex); logs_queue_ptr = logs_queue; @@ -97,7 +96,7 @@ protected: QueryStatus * parent_query_, ProfileEvents::Counters * parent_counters, MemoryTracker * parent_memory_tracker, - const SystemLogsQueueWeakPtr & logs_queue_ptr_, + const InternalTextLogsQueueWeakPtr & logs_queue_ptr_, bool check_detached = true); void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false); @@ -117,7 +116,7 @@ protected: Context * query_context = nullptr; /// A logs queue used by TCPHandler to pass logs to a client - SystemLogsQueueWeakPtr logs_queue_ptr; + InternalTextLogsQueueWeakPtr logs_queue_ptr; UInt64 query_start_time_nanoseconds = 0; time_t query_start_time = 0; diff --git a/dbms/src/DataStreams/SystemLogsRowOutputStream.cpp b/dbms/src/DataStreams/InternalTextLogsRowOutputStream.cpp similarity index 83% rename from dbms/src/DataStreams/SystemLogsRowOutputStream.cpp rename to dbms/src/DataStreams/InternalTextLogsRowOutputStream.cpp index 10ca4c30235..02d2f8cf440 100644 --- a/dbms/src/DataStreams/SystemLogsRowOutputStream.cpp +++ b/dbms/src/DataStreams/InternalTextLogsRowOutputStream.cpp @@ -1,6 +1,6 @@ -#include "SystemLogsRowOutputStream.h" +#include "InternalTextLogsRowOutputStream.h" #include -#include +#include #include #include #include @@ -11,12 +11,12 @@ namespace DB { -Block SystemLogsRowOutputStream::getHeader() const +Block InternalTextLogsRowOutputStream::getHeader() const { - return SystemLogsQueue::getSampleBlock(); + return InternalTextLogsQueue::getSampleBlock(); } -void SystemLogsRowOutputStream::write(const Block & block) +void InternalTextLogsRowOutputStream::write(const Block & block) { auto & array_event_time = typeid_cast(*block.getByName("event_time").column).getData(); auto & array_microseconds = typeid_cast(*block.getByName("event_time_microseconds").column).getData(); @@ -31,6 +31,14 @@ void SystemLogsRowOutputStream::write(const Block & block) for (size_t row_num = 0; row_num < block.rows(); ++row_num) { + auto host_name = column_host_name.getDataAt(row_num); + if (host_name.size) + { + writeCString("[", wb); + writeString(host_name, wb); + writeCString("] ", wb); + } + auto event_time = array_event_time[row_num]; writeDateTimeText<'.', ':'>(event_time, wb); @@ -43,20 +51,12 @@ void SystemLogsRowOutputStream::write(const Block & block) writeChar('0' + ((microseconds / 10) % 10), wb); writeChar('0' + ((microseconds / 1) % 10), wb); - auto host_name = column_host_name.getDataAt(row_num); - if (host_name.size) - { - writeCString(" [ ", wb); - writeString(host_name, wb); - writeCString(" ]", wb); - } - auto query_id = column_query_id.getDataAt(row_num); if (query_id.size) { - writeCString(" { ", wb); + writeCString(" {", wb); writeString(query_id, wb); - writeCString(" }", wb); + writeCString("}", wb); } UInt32 thread_number = array_thread_number[row_num]; @@ -65,7 +65,7 @@ void SystemLogsRowOutputStream::write(const Block & block) writeCString(" ] <", wb); Int8 priority = array_priority[row_num]; - writeString(SystemLogsQueue::getPriorityName(priority), wb); + writeString(InternalTextLogsQueue::getPriorityName(priority), wb); writeCString("> ", wb); auto source = column_source.getDataAt(row_num); diff --git a/dbms/src/DataStreams/SystemLogsRowOutputStream.h b/dbms/src/DataStreams/InternalTextLogsRowOutputStream.h similarity index 77% rename from dbms/src/DataStreams/SystemLogsRowOutputStream.h rename to dbms/src/DataStreams/InternalTextLogsRowOutputStream.h index 217209da608..3f54a00e633 100644 --- a/dbms/src/DataStreams/SystemLogsRowOutputStream.h +++ b/dbms/src/DataStreams/InternalTextLogsRowOutputStream.h @@ -9,11 +9,11 @@ namespace DB /// Prints internal server logs /// Input blocks have to have the same structure as SystemLogsQueue::getSampleBlock() /// NOTE: IRowOutputStream does not suite well for this case -class SystemLogsRowOutputStream : public IBlockOutputStream +class InternalTextLogsRowOutputStream : public IBlockOutputStream { public: - SystemLogsRowOutputStream(WriteBuffer & buf_out) : wb(buf_out) {} + InternalTextLogsRowOutputStream(WriteBuffer & buf_out) : wb(buf_out) {} Block getHeader() const override; diff --git a/dbms/src/Interpreters/ClickHouseLogChannel.cpp b/dbms/src/Interpreters/ClickHouseLogChannel.cpp deleted file mode 100644 index be7dc891032..00000000000 --- a/dbms/src/Interpreters/ClickHouseLogChannel.cpp +++ /dev/null @@ -1,49 +0,0 @@ -#include "ClickHouseLogChannel.h" - -#include -#include -#include -#include - -#include -#include - -#include -#include - - -namespace DB -{ - - -void ClickHouseLogChannel::log(const Poco::Message & msg) -{ - if (auto logs_queue = CurrentThread::getSystemLogsQueue()) - { - /// Too noisy message - if (msg.getPriority() > logs_queue->max_priority) - return; - - MutableColumns columns = SystemLogsQueue::getSampleColumns(); - - /// TODO: it would be better if the time was exactly the same as one in OwnPatternFormatter - ::timeval tv; - if (0 != gettimeofday(&tv, nullptr)) - DB::throwFromErrno("Cannot gettimeofday"); - - size_t i = 0; - columns[i++]->insert(static_cast(tv.tv_sec)); - columns[i++]->insert(static_cast(tv.tv_usec)); - columns[i++]->insert(DNSResolver::instance().getHostName()); - columns[i++]->insert(CurrentThread::getCurrentQueryID()); - columns[i++]->insert(static_cast(Poco::ThreadNumber::get())); - columns[i++]->insert(static_cast(msg.getPriority())); - columns[i++]->insert(msg.getSource()); - columns[i++]->insert(msg.getText()); - - logs_queue->emplace(std::move(columns)); - } -} - - -} diff --git a/dbms/src/Interpreters/ClickHouseLogChannel.h b/dbms/src/Interpreters/ClickHouseLogChannel.h deleted file mode 100644 index e1104476a05..00000000000 --- a/dbms/src/Interpreters/ClickHouseLogChannel.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once -#include - - -namespace DB -{ - -/// Poco::Channel used to implement passing of query logs to Client via TCP interface -class ClickHouseLogChannel : public Poco::Channel -{ -public: - ClickHouseLogChannel() = default; - - void log(const Poco::Message & msg) override; -}; - -} diff --git a/dbms/src/Core/SystemLogsQueue.cpp b/dbms/src/Interpreters/InternalTextLogsQueue.cpp similarity index 83% rename from dbms/src/Core/SystemLogsQueue.cpp rename to dbms/src/Interpreters/InternalTextLogsQueue.cpp index 54b02be7fc5..552b98652ee 100644 --- a/dbms/src/Core/SystemLogsQueue.cpp +++ b/dbms/src/Interpreters/InternalTextLogsQueue.cpp @@ -1,4 +1,4 @@ -#include "SystemLogsQueue.h" +#include "InternalTextLogsQueue.h" #include #include #include @@ -10,12 +10,12 @@ namespace DB { -SystemLogsQueue::SystemLogsQueue() +InternalTextLogsQueue::InternalTextLogsQueue() : ConcurrentBoundedQueue(std::numeric_limits::max()), max_priority(Poco::Message::Priority::PRIO_INFORMATION) {} -Block SystemLogsQueue::getSampleBlock() +Block InternalTextLogsQueue::getSampleBlock() { return Block { {std::make_shared(), "event_time"}, @@ -29,13 +29,13 @@ Block SystemLogsQueue::getSampleBlock() }; } -MutableColumns SystemLogsQueue::getSampleColumns() +MutableColumns InternalTextLogsQueue::getSampleColumns() { static Block sample_block = getSampleBlock(); return sample_block.cloneEmptyColumns(); } -const char * SystemLogsQueue::getPriorityName(int priority) +const char * InternalTextLogsQueue::getPriorityName(int priority) { /// See Poco::Message::Priority diff --git a/dbms/src/Core/SystemLogsQueue.h b/dbms/src/Interpreters/InternalTextLogsQueue.h similarity index 59% rename from dbms/src/Core/SystemLogsQueue.h rename to dbms/src/Interpreters/InternalTextLogsQueue.h index 10535f245f9..94992bda311 100644 --- a/dbms/src/Core/SystemLogsQueue.h +++ b/dbms/src/Interpreters/InternalTextLogsQueue.h @@ -1,18 +1,18 @@ #pragma once -#include -#include +#include "../Common/ConcurrentBoundedQueue.h" +#include "../Core/Block.h" namespace DB { -class SystemLogsQueue : public ConcurrentBoundedQueue +class InternalTextLogsQueue : public ConcurrentBoundedQueue { public: /// You should not push logs in the queue if their priority greater max_priority int max_priority; - SystemLogsQueue(); + InternalTextLogsQueue(); static Block getSampleBlock(); static MutableColumns getSampleColumns(); @@ -21,7 +21,7 @@ public: static const char * getPriorityName(int priority); }; -using SystemLogsQueuePtr = std::shared_ptr; +using InternalTextLogsQueuePtr = std::shared_ptr; } diff --git a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh index a6780ef7a5d..443026f5fa6 100755 --- a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh +++ b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh @@ -103,4 +103,4 @@ WHERE # Clean -#rm "$server_logs_file" +rm "$server_logs_file" diff --git a/libs/libdaemon/CMakeLists.txt b/libs/libdaemon/CMakeLists.txt index a0e4e7d2733..1bd7731664a 100644 --- a/libs/libdaemon/CMakeLists.txt +++ b/libs/libdaemon/CMakeLists.txt @@ -1,11 +1,17 @@ add_library (daemon ${SPLIT_SHARED} src/BaseDaemon.cpp src/GraphiteWriter.cpp + src/ExtendedLogChannel.cpp src/OwnPatternFormatter.cpp + src/OwnFormattingChannel.cpp + src/OwnSplitChannel.cpp include/daemon/BaseDaemon.h include/daemon/GraphiteWriter.h + include/daemon/ExtendedLogChannel.h include/daemon/OwnPatternFormatter.h + include/daemon/OwnFormattingChannel.h + include/daemon/OwnSplitChannel.h ) if (USE_UNWIND) diff --git a/libs/libdaemon/include/daemon/ExtendedLogChannel.h b/libs/libdaemon/include/daemon/ExtendedLogChannel.h new file mode 100644 index 00000000000..6d03970676a --- /dev/null +++ b/libs/libdaemon/include/daemon/ExtendedLogChannel.h @@ -0,0 +1,39 @@ +#pragma once +#include +#include + + +namespace DB +{ + +/// Poco::Message with more ClickHouse-specific info +/// NOTE: Poco::Message is not polymorphic class, so we can't use inheritance in couple with dynamic_cast<>() +class ExtendedLogMessage +{ +public: + + explicit ExtendedLogMessage(const Poco::Message & base) : base(base) {} + + /// Attach additional data to the message + static ExtendedLogMessage getFrom(const Poco::Message & base); + + // Do not copy for efficiency reasons + const Poco::Message & base; + + UInt32 time_seconds = 0; + UInt32 time_microseconds = 0; + + UInt32 thread_number = 0; + std::string query_id; +}; + + +/// Interface extension of Poco::Channel +class ExtendedLogChannel +{ +public: + virtual void logExtended(const ExtendedLogMessage & msg) = 0; + virtual ~ExtendedLogChannel() = default; +}; + +} diff --git a/libs/libdaemon/include/daemon/OwnFormattingChannel.h b/libs/libdaemon/include/daemon/OwnFormattingChannel.h new file mode 100644 index 00000000000..bcce42df866 --- /dev/null +++ b/libs/libdaemon/include/daemon/OwnFormattingChannel.h @@ -0,0 +1,57 @@ +#pragma once +#include +#include +#include +#include + + +class OwnPatternFormatter; + + +namespace DB +{ + + +// Like Poco::FormattingChannel but supports the extended logging interface and log level filter +class OwnFormattingChannel : public Poco::Channel, public ExtendedLogChannel +{ +public: + + explicit OwnFormattingChannel(Poco::AutoPtr pFormatter_ = nullptr, Poco::AutoPtr pChannel_ = nullptr) + : pFormatter(std::move(pFormatter_)), pChannel(std::move(pChannel_)) {} + + void setChannel(Poco::AutoPtr pChannel_) + { + pChannel = std::move(pChannel_); + } + + void setLevel(Poco::Message::Priority priority_) + { + priority = priority_; + } + + void open() override + { + if (pChannel) + pChannel->open(); + } + + void close() override + { + if (pChannel) + pChannel->close(); + } + + void log(const Poco::Message & msg) override; + void logExtended(const ExtendedLogMessage & msg) override; + + ~OwnFormattingChannel() override; + +private: + + Poco::AutoPtr pFormatter; + Poco::AutoPtr pChannel; + Poco::Message::Priority priority = Poco::Message::PRIO_TRACE; +}; + +} diff --git a/libs/libdaemon/include/daemon/OwnPatternFormatter.h b/libs/libdaemon/include/daemon/OwnPatternFormatter.h index 11c195f2171..e90734c21e8 100644 --- a/libs/libdaemon/include/daemon/OwnPatternFormatter.h +++ b/libs/libdaemon/include/daemon/OwnPatternFormatter.h @@ -2,6 +2,7 @@ #include +#include /** Форматирует по своему. @@ -31,9 +32,10 @@ public: ADD_LAYER_TAG = 1 << 0 }; - OwnPatternFormatter(const BaseDaemon * daemon_, Options options_ = ADD_NOTHING) : Poco::PatternFormatter(""), daemon(daemon_), options(options_) {} + OwnPatternFormatter(const BaseDaemon * daemon_, Options options_ = ADD_NOTHING); void format(const Poco::Message & msg, std::string & text) override; + void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text); private: const BaseDaemon * daemon; diff --git a/libs/libdaemon/include/daemon/OwnSplitChannel.h b/libs/libdaemon/include/daemon/OwnSplitChannel.h new file mode 100644 index 00000000000..0f0702c2ab5 --- /dev/null +++ b/libs/libdaemon/include/daemon/OwnSplitChannel.h @@ -0,0 +1,35 @@ +#pragma once +#include +#include +#include +#include + + +namespace DB +{ + +/// Works as Poco::SplitterChannel, but performs additional work: +/// passes logs to Client via TCP interface +/// tries to use extended logging interface of child for more comprehensive logging +class OwnSplitChannel : public Poco::Channel +{ +public: + OwnSplitChannel() = default; + + /// Makes an extended message from msg and passes it to the client logs queue and child (if possible) + void log(const Poco::Message & msg) override; + + /// Adds a child channel + void addChannel(Poco::AutoPtr channel); + + ~OwnSplitChannel() = default; + +private: + + using ChannelPtr = Poco::AutoPtr; + /// Handler and its pointer casted to extended interface + using ExtendedChannelPtrPair = std::pair; + std::vector channels; +}; + +} diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index c391418bc3f..f5aae3f528e 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -1,7 +1,9 @@ #include +#include +#include #include -#include +#include #include #include @@ -705,8 +707,9 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) bool is_daemon = config.getBool("application.runAsDaemon", false); - // Split log and error log. - Poco::AutoPtr split = new SplitterChannel; + /// Split logs to ordinary log, error log, syslog and console. + /// Use extended interface of Channel for more comprehensive logging. + Poco::AutoPtr split = new DB::OwnSplitChannel; auto log_level = config.getString("logger.level", "trace"); const auto log_path = config.getString("logger.log", ""); @@ -716,10 +719,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) std::cerr << "Logging " << log_level << " to " << log_path << std::endl; // Set up two channel chains. - Poco::AutoPtr pf = new OwnPatternFormatter(this); - pf->setProperty("times", "local"); - Poco::AutoPtr log = new FormattingChannel(pf); - log_file = new FileChannel; + Poco::AutoPtr log_file = new FileChannel; log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(log_path).absolute().toString()); log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); @@ -727,9 +727,12 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) log_file->setProperty(Poco::FileChannel::PROP_PURGECOUNT, config.getRawString("logger.count", "1")); log_file->setProperty(Poco::FileChannel::PROP_FLUSH, config.getRawString("logger.flush", "true")); log_file->setProperty(Poco::FileChannel::PROP_ROTATEONOPEN, config.getRawString("logger.rotateOnOpen", "false")); - log->setChannel(log_file); - split->addChannel(log); log_file->open(); + + Poco::AutoPtr pf = new OwnPatternFormatter(this); + + Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, log_file); + split->addChannel(log); } const auto errorlog_path = config.getString("logger.errorlog", ""); @@ -737,12 +740,8 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) { createDirectory(errorlog_path); std::cerr << "Logging errors to " << errorlog_path << std::endl; - Poco::AutoPtr level = new Poco::LevelFilterChannel; - level->setLevel(Message::PRIO_NOTICE); - Poco::AutoPtr pf = new OwnPatternFormatter(this); - pf->setProperty("times", "local"); - Poco::AutoPtr errorlog = new FormattingChannel(pf); - error_log_file = new FileChannel; + + Poco::AutoPtr error_log_file = new FileChannel; error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(errorlog_path).absolute().toString()); error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); @@ -750,10 +749,13 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) error_log_file->setProperty(Poco::FileChannel::PROP_PURGECOUNT, config.getRawString("logger.count", "1")); error_log_file->setProperty(Poco::FileChannel::PROP_FLUSH, config.getRawString("logger.flush", "true")); error_log_file->setProperty(Poco::FileChannel::PROP_ROTATEONOPEN, config.getRawString("logger.rotateOnOpen", "false")); - errorlog->setChannel(error_log_file); - level->setChannel(errorlog); - split->addChannel(level); + + Poco::AutoPtr pf = new OwnPatternFormatter(this); + + Poco::AutoPtr errorlog = new DB::OwnFormattingChannel(pf, error_log_file); + errorlog->setLevel(Message::PRIO_NOTICE); errorlog->open(); + split->addChannel(errorlog); } /// "dynamic_layer_selection" is needed only for Yandex.Metrika, that share part of ClickHouse code. @@ -761,10 +763,6 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) if (config.getBool("logger.use_syslog", false) || config.getBool("dynamic_layer_selection", false)) { - Poco::AutoPtr pf = new OwnPatternFormatter(this, OwnPatternFormatter::ADD_LAYER_TAG); - pf->setProperty("times", "local"); - Poco::AutoPtr log = new FormattingChannel(pf); - const std::string & cmd_name = commandName(); if (config.has("logger.syslog.address")) @@ -786,28 +784,21 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) syslog_channel->setProperty(Poco::SyslogChannel::PROP_OPTIONS, config.getString("logger.syslog.options", "LOG_CONS|LOG_PID")); syslog_channel->setProperty(Poco::SyslogChannel::PROP_FACILITY, config.getString("logger.syslog.facility", "LOG_DAEMON")); } - - log->setChannel(syslog_channel); - split->addChannel(log); syslog_channel->open(); + + Poco::AutoPtr pf = new OwnPatternFormatter(this, OwnPatternFormatter::ADD_LAYER_TAG); + + Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, syslog_channel); + split->addChannel(log); } if (config.getBool("logger.console", false) || (!config.hasProperty("logger.console") && !is_daemon && (isatty(STDIN_FILENO) || isatty(STDERR_FILENO)))) { - Poco::AutoPtr file = new ConsoleChannel; - Poco::AutoPtr pf = new OwnPatternFormatter(this); - pf->setProperty("times", "local"); - Poco::AutoPtr log = new FormattingChannel(pf); - log->setChannel(file); + Poco::AutoPtr log = new DB::OwnFormattingChannel(new OwnPatternFormatter(this), new Poco::ConsoleChannel); logger().warning("Logging " + log_level + " to console"); split->addChannel(log); } - { - Poco::AutoPtr internal_logger = new DB::ClickHouseLogChannel; - split->addChannel(internal_logger); - } - split->open(); logger().close(); logger().setChannel(split); diff --git a/libs/libdaemon/src/ExtendedLogChannel.cpp b/libs/libdaemon/src/ExtendedLogChannel.cpp new file mode 100644 index 00000000000..94413f8b8ed --- /dev/null +++ b/libs/libdaemon/src/ExtendedLogChannel.cpp @@ -0,0 +1,27 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +ExtendedLogMessage ExtendedLogMessage::getFrom(const Poco::Message & base) +{ + ExtendedLogMessage msg_ext(base); + + ::timeval tv; + if (0 != gettimeofday(&tv, nullptr)) + DB::throwFromErrno("Cannot gettimeofday"); + + msg_ext.time_seconds = static_cast(tv.tv_sec); + msg_ext.time_microseconds = static_cast(tv.tv_usec); + msg_ext.query_id = CurrentThread::getCurrentQueryID(); + msg_ext.thread_number = Poco::ThreadNumber::get(); + + return msg_ext; +} + +} diff --git a/libs/libdaemon/src/OwnFormattingChannel.cpp b/libs/libdaemon/src/OwnFormattingChannel.cpp new file mode 100644 index 00000000000..a91f96bf650 --- /dev/null +++ b/libs/libdaemon/src/OwnFormattingChannel.cpp @@ -0,0 +1,33 @@ +#include +#include + + +namespace DB +{ + +void OwnFormattingChannel::logExtended(const ExtendedLogMessage & msg) +{ + if (pChannel && priority >= msg.base.getPriority()) + { + if (pFormatter) + { + std::string text; + pFormatter->formatExtended(msg, text); + pChannel->log(Poco::Message(msg.base, text)); + } + else + { + pChannel->log(msg.base); + } + } +} + +void OwnFormattingChannel::log(const Poco::Message & msg) +{ + logExtended(ExtendedLogMessage::getFrom(msg)); +} + +OwnFormattingChannel::~OwnFormattingChannel() = default; + + +} diff --git a/libs/libdaemon/src/OwnPatternFormatter.cpp b/libs/libdaemon/src/OwnPatternFormatter.cpp index cb0c745a375..310cc1d9fb9 100644 --- a/libs/libdaemon/src/OwnPatternFormatter.cpp +++ b/libs/libdaemon/src/OwnPatternFormatter.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include @@ -12,10 +12,16 @@ #include -void OwnPatternFormatter::format(const Poco::Message & msg, std::string & text) +OwnPatternFormatter::OwnPatternFormatter(const BaseDaemon * daemon_, OwnPatternFormatter::Options options_) + : Poco::PatternFormatter(""), daemon(daemon_), options(options_) {} + + +void OwnPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) { DB::WriteBufferFromString wb(text); + const Poco::Message & msg = msg_ext.base; + /// For syslog: tag must be before message and first whitespace. if (options & ADD_LAYER_TAG && daemon) { @@ -28,24 +34,26 @@ void OwnPatternFormatter::format(const Poco::Message & msg, std::string & text) } } - /// Output time with microsecond resolution. - ::timeval tv; - if (0 != gettimeofday(&tv, nullptr)) - DB::throwFromErrno("Cannot gettimeofday"); - /// Change delimiters in date for compatibility with old logs. - DB::writeDateTimeText<'.', ':'>(tv.tv_sec, wb); + DB::writeDateTimeText<'.', ':'>(msg_ext.time_seconds, wb); DB::writeChar('.', wb); - DB::writeChar('0' + ((tv.tv_usec / 100000) % 10), wb); - DB::writeChar('0' + ((tv.tv_usec / 10000) % 10), wb); - DB::writeChar('0' + ((tv.tv_usec / 1000) % 10), wb); - DB::writeChar('0' + ((tv.tv_usec / 100) % 10), wb); - DB::writeChar('0' + ((tv.tv_usec / 10) % 10), wb); - DB::writeChar('0' + ((tv.tv_usec / 1) % 10), wb); + DB::writeChar('0' + ((msg_ext.time_microseconds / 100000) % 10), wb); + DB::writeChar('0' + ((msg_ext.time_microseconds / 10000) % 10), wb); + DB::writeChar('0' + ((msg_ext.time_microseconds / 1000) % 10), wb); + DB::writeChar('0' + ((msg_ext.time_microseconds / 100) % 10), wb); + DB::writeChar('0' + ((msg_ext.time_microseconds / 10) % 10), wb); + DB::writeChar('0' + ((msg_ext.time_microseconds / 1) % 10), wb); + + if (!msg_ext.query_id.empty()) + { + writeCString(" {", wb); + DB::writeString(msg_ext.query_id, wb); + writeCString("}", wb); + } writeCString(" [ ", wb); - DB::writeIntText(Poco::ThreadNumber::get(), wb); + DB::writeIntText(msg_ext.thread_number, wb); writeCString(" ] <", wb); DB::writeString(getPriorityName(static_cast(msg.getPriority())), wb); writeCString("> ", wb); @@ -53,3 +61,8 @@ void OwnPatternFormatter::format(const Poco::Message & msg, std::string & text) writeCString(": ", wb); DB::writeString(msg.getText(), wb); } + +void OwnPatternFormatter::format(const Poco::Message & msg, std::string & text) +{ + formatExtended(DB::ExtendedLogMessage::getFrom(msg), text); +} diff --git a/libs/libdaemon/src/OwnSplitChannel.cpp b/libs/libdaemon/src/OwnSplitChannel.cpp new file mode 100644 index 00000000000..03c9a77a7c8 --- /dev/null +++ b/libs/libdaemon/src/OwnSplitChannel.cpp @@ -0,0 +1,62 @@ +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include + + +namespace DB +{ + + +void OwnSplitChannel::log(const Poco::Message & msg) +{ + auto logs_queue = CurrentThread::getInternalTextLogsQueue(); + + if (channels.empty() && (logs_queue == nullptr || msg.getPriority() > logs_queue->max_priority)) + return; + + ExtendedLogMessage msg_ext = ExtendedLogMessage::getFrom(msg); + + /// Log data to child channels + for (auto & channel : channels) + { + if (channel.second) + channel.second->logExtended(msg_ext); // extended child + else + channel.first->log(msg); // ordinary child + } + + /// Log to "TCP queue" if message is not too noisy + if (logs_queue && msg.getPriority() <= logs_queue->max_priority) + { + MutableColumns columns = InternalTextLogsQueue::getSampleColumns(); + + size_t i = 0; + columns[i++]->insert(static_cast(msg_ext.time_seconds)); + columns[i++]->insert(static_cast(msg_ext.time_microseconds)); + columns[i++]->insert(DNSResolver::instance().getHostName()); + columns[i++]->insert(msg_ext.query_id); + columns[i++]->insert(static_cast(msg_ext.thread_number)); + columns[i++]->insert(static_cast(msg.getPriority())); + columns[i++]->insert(msg.getSource()); + columns[i++]->insert(msg.getText()); + + logs_queue->emplace(std::move(columns)); + } +} + +void OwnSplitChannel::addChannel(Poco::AutoPtr channel) +{ + channels.emplace_back(std::move(channel), dynamic_cast(channel.get())); +} + + +} From e031b5360afc3c331c8df0703ed0fe06bc3084b8 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Mon, 18 Jun 2018 16:50:20 +0300 Subject: [PATCH 086/315] Minor enhancement. [#CLICKHOUSE-2910] --- dbms/src/Common/TaskStatsInfoGetter.cpp | 27 +++++++++---------------- 1 file changed, 10 insertions(+), 17 deletions(-) diff --git a/dbms/src/Common/TaskStatsInfoGetter.cpp b/dbms/src/Common/TaskStatsInfoGetter.cpp index 04715f7d1ca..9c7f2a224b5 100644 --- a/dbms/src/Common/TaskStatsInfoGetter.cpp +++ b/dbms/src/Common/TaskStatsInfoGetter.cpp @@ -244,25 +244,18 @@ int TaskStatsInfoGetter::getDefaultTid() return default_tid; } +static bool tryGetTaskStats() +{ + TaskStatsInfoGetter getter; + ::taskstats stat; + return getter.tryGetStat(stat); +} + bool TaskStatsInfoGetter::checkProcessHasRequiredPermissions() { - /// 0 - wasn't checked - /// 1 - checked, has no permissions - /// 2 - checked, has permissions - static std::atomic premissions_check_status{0}; - - int status = premissions_check_status.load(std::memory_order_relaxed); - - if (status == 0) - { - TaskStatsInfoGetter getter; - ::taskstats stat; - - status = getter.tryGetStat(stat) ? 2 : 1; - premissions_check_status.store(status, std::memory_order_relaxed); - } - - return status == 2; + /// It is thread- and exception- safe since C++11 + static bool res = tryGetTaskStats(); + return res; } } From 8dc1bebf0080999388d8c8189b840e4cfc099ef7 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Mon, 18 Jun 2018 19:30:26 +0300 Subject: [PATCH 087/315] Add logs forwarding from remote severs to client. [#CLICKHOUSE-2910] --- .../DataStreams/RemoteBlockInputStream.cpp | 5 +++ .../DataStreams/RemoteBlockOutputStream.cpp | 6 +++- .../Interpreters/InternalTextLogsQueue.cpp | 11 +++++++ dbms/src/Interpreters/InternalTextLogsQueue.h | 7 ++-- ...4_performance_introspection_and_logging.sh | 33 +++++++++++++++++-- 5 files changed, 57 insertions(+), 5 deletions(-) diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 157173f5a08..49497c0cedb 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -1,8 +1,10 @@ #include #include #include +#include #include #include +#include #include @@ -233,6 +235,9 @@ Block RemoteBlockInputStream::readImpl() break; case Protocol::Server::Log: + /// Pass logs from remote server to client + if (auto log_queue = CurrentThread::getInternalTextLogsQueue()) + log_queue->pushBlock(std::move(packet.block)); break; default: diff --git a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp index 9ccb0507735..f1e68a6a0c1 100644 --- a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp @@ -4,6 +4,8 @@ #include #include +#include +#include namespace DB @@ -43,7 +45,9 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const } else if (Protocol::Server::Log == packet.type) { - /// Do nothing + /// Pass logs from remote server to client + if (auto log_queue = CurrentThread::getInternalTextLogsQueue()) + log_queue->pushBlock(std::move(packet.block)); } else throw NetException("Unexpected packet from server (expected Data or Exception, got " diff --git a/dbms/src/Interpreters/InternalTextLogsQueue.cpp b/dbms/src/Interpreters/InternalTextLogsQueue.cpp index 552b98652ee..8f64cfc7094 100644 --- a/dbms/src/Interpreters/InternalTextLogsQueue.cpp +++ b/dbms/src/Interpreters/InternalTextLogsQueue.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -35,6 +36,16 @@ MutableColumns InternalTextLogsQueue::getSampleColumns() return sample_block.cloneEmptyColumns(); } +void InternalTextLogsQueue::pushBlock(Block && log_block) +{ + static Block sample_block = getSampleBlock(); + + if (blocksHaveEqualStructure(sample_block, log_block)) + emplace(log_block.mutateColumns()); + else + LOG_WARNING(&Poco::Logger::get("InternalTextLogsQueue"), "Log block have different structure"); +} + const char * InternalTextLogsQueue::getPriorityName(int priority) { /// See Poco::Message::Priority diff --git a/dbms/src/Interpreters/InternalTextLogsQueue.h b/dbms/src/Interpreters/InternalTextLogsQueue.h index 94992bda311..42711181a1a 100644 --- a/dbms/src/Interpreters/InternalTextLogsQueue.h +++ b/dbms/src/Interpreters/InternalTextLogsQueue.h @@ -1,6 +1,6 @@ #pragma once -#include "../Common/ConcurrentBoundedQueue.h" -#include "../Core/Block.h" +#include +#include namespace DB @@ -17,6 +17,9 @@ public: static Block getSampleBlock(); static MutableColumns getSampleColumns(); + /// Is used to pass block from remote server to the client + void pushBlock(Block && log_block); + /// Converts priority from Poco::Message::Priority to a string static const char * getPriorityName(int priority); }; diff --git a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh index 443026f5fa6..3f12c151e3e 100755 --- a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh +++ b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh @@ -46,7 +46,7 @@ SELECT threads_realtime >= threads_time_user_system_io, any(length(thread_numbers)) >= 1 FROM - (SELECT * FROM system.query_log WHERE event_date >= today()-1 AND type=2 AND query='$heavy_cpu_query' ORDER BY event_time DESC LIMIT 1) + (SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-1 AND type=2 ORDER BY event_time DESC LIMIT 1) ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV" @@ -71,7 +71,7 @@ FROM sumIf(PV, PN = 'RealTimeMicroseconds') AS thread_realtime, sumIf(PV, PN IN ('UserTimeMicroseconds', 'SystemTimeMicroseconds', 'OSIOWaitMicroseconds')) AS thread_time_user_system_io FROM - (SELECT * FROM system.query_thread_log WHERE event_date >= today()-1 AND query_id='$query_id') + (SELECT * FROM system.query_thread_log PREWHERE query_id='$query_id' WHERE event_date >= today()-1) ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV GROUP BY thread_number ) @@ -102,5 +102,34 @@ WHERE " +# Check that logs from remote servers are passed from client + +# SELECT +> "$server_logs_file" +$CLICKHOUSE_CLIENT $settings -q "SELECT 1 FROM system.one FORMAT Null" +lines_one_server=`cat "$server_logs_file" | wc -l` + +> "$server_logs_file" +$CLICKHOUSE_CLIENT $settings -q "SELECT 1 FROM remote('127.0.0.2,127.0.0.3', system, one) FORMAT Null" +lines_two_servers=`cat "$server_logs_file" | wc -l` + +(( $lines_two_servers >= 2 * $lines_one_server )) || echo "Fail: $lines_two_servers $lines_one_server" + +# INSERT +$CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS test.null" +$CLICKHOUSE_CLIENT $settings -q "CREATE TABLE test.null (i Int8) ENGINE = Null" + +> "$server_logs_file" +$CLICKHOUSE_CLIENT $settings -q "INSERT INTO test.null VALUES (0)" +lines_one_server=`cat "$server_logs_file" | wc -l` + +> "$server_logs_file" +$CLICKHOUSE_CLIENT $settings -q "INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'null') VALUES (0)" +lines_two_servers=`cat "$server_logs_file" | wc -l` + +$CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS test.null" +(( $lines_two_servers > $lines_one_server )) || echo "Fail: $lines_two_servers $lines_one_server" + + # Clean rm "$server_logs_file" From e13ba090041e6714167a39c62a211d7edfd32060 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Tue, 19 Jun 2018 23:30:35 +0300 Subject: [PATCH 088/315] Rewritten ThreadStatus via adding shared thread state. [#CLICKHOUSE-2910] Fixed race condition in SystemLog. --- dbms/programs/server/HTTPHandler.cpp | 14 +- dbms/programs/server/TCPHandler.cpp | 5 +- dbms/src/Common/BackgroundSchedulePool.cpp | 16 +- dbms/src/Common/BackgroundSchedulePool.h | 4 + dbms/src/Common/CurrentThread.cpp | 88 ++----- dbms/src/Common/CurrentThread.h | 23 +- dbms/src/Common/MemoryTracker.cpp | 2 +- dbms/src/Common/ThreadStatus.cpp | 249 +++++++++++------- dbms/src/Common/ThreadStatus.h | 90 ++++--- .../AsynchronousBlockInputStream.cpp | 4 +- ...regatedMemoryEfficientBlockInputStream.cpp | 23 +- ...ggregatedMemoryEfficientBlockInputStream.h | 4 +- .../src/DataStreams/ParallelInputsProcessor.h | 12 +- dbms/src/Interpreters/Aggregator.cpp | 18 +- dbms/src/Interpreters/ProcessList.cpp | 74 +++--- dbms/src/Interpreters/ProcessList.h | 17 +- dbms/src/Interpreters/SystemLog.h | 27 +- dbms/src/Interpreters/executeQuery.cpp | 8 +- .../DistributedBlockOutputStream.cpp | 6 +- .../MergeTree/BackgroundProcessingPool.cpp | 14 +- .../MergeTree/BackgroundProcessingPool.h | 4 + 21 files changed, 399 insertions(+), 303 deletions(-) diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 8be79a749dc..ce5b1416720 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -209,7 +209,13 @@ void HTTPHandler::processQuery( Poco::Net::HTTPServerResponse & response, Output & used_output) { + Context context = server.context(); + context.setGlobalContext(server.context()); + CurrentThread::initializeQuery(); + /// It will forcibly detach query even if unexpected error ocurred and detachQuery() was not called + /// Normal detaching is happen in BlockIO callbacks + SCOPE_EXIT({CurrentThread::detachQueryIfNotDetached();}); LOG_TRACE(log, "Request URI: " << request.getURI()); @@ -260,14 +266,9 @@ void HTTPHandler::processQuery( } std::string query_id = params.get("query_id", ""); - - const auto & config = server.config(); - - Context context = server.context(); - context.setGlobalContext(server.context()); - context.setUser(user, password, request.clientAddress(), quota_key); context.setCurrentQueryId(query_id); + CurrentThread::attachQueryContext(context); /// The user could specify session identifier and session timeout. /// It allows to modify settings, create temporary tables and reuse them in subsequent requests. @@ -276,6 +277,7 @@ void HTTPHandler::processQuery( String session_id; std::chrono::steady_clock::duration session_timeout; bool session_is_set = params.has("session_id"); + const auto & config = server.config(); if (session_is_set) { diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 28983b2eea3..40971486c8a 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -160,7 +160,7 @@ void TCPHandler::runImpl() { state.logs_queue = std::make_shared(); state.logs_queue->max_priority = Poco::Logger::parseLevel(query_context.getSettingsRef().server_logs_level.value); - CurrentThread::attachSystemLogsQueue(state.logs_queue); + CurrentThread::attachInternalTextLogsQueue(state.logs_queue); } query_context.setExternalTablesInitializer([&global_settings, this] (Context & context) { @@ -264,6 +264,9 @@ void TCPHandler::runImpl() try { + /// It will forcibly detach query even if unexpected error ocсurred and detachQuery() was not called + CurrentThread::detachQueryIfNotDetached(); + state.reset(); } catch (...) diff --git a/dbms/src/Common/BackgroundSchedulePool.cpp b/dbms/src/Common/BackgroundSchedulePool.cpp index 594dffcedd1..d45446ae455 100644 --- a/dbms/src/Common/BackgroundSchedulePool.cpp +++ b/dbms/src/Common/BackgroundSchedulePool.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -141,6 +142,12 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size) { LOG_INFO(&Logger::get("BackgroundSchedulePool"), "Create BackgroundSchedulePool with " << size << " threads"); + /// Put all threads of both thread pools to one thread group + /// The master thread exits immediately + CurrentThread::initializeQuery(); + thread_group = CurrentThread::getGroup(); + CurrentThread::detachQuery(); + threads.resize(size); for (auto & thread : threads) thread = std::thread([this] { threadFunction(); }); @@ -213,7 +220,10 @@ void BackgroundSchedulePool::threadFunction() { setThreadName("BackgrSchedPool"); - CurrentThread::attachQuery(nullptr); + /// Put all threads to one thread pool + CurrentThread::attachTo(thread_group); + SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); }); + CurrentThread::getMemoryTracker().setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool); while (!shutdown) @@ -231,6 +241,10 @@ void BackgroundSchedulePool::delayExecutionThreadFunction() { setThreadName("BckSchPoolDelay"); + /// Put all threads to one thread pool + CurrentThread::attachTo(thread_group); + SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); }); + while (!shutdown) { TaskInfoPtr task; diff --git a/dbms/src/Common/BackgroundSchedulePool.h b/dbms/src/Common/BackgroundSchedulePool.h index 2fda381b111..bfd80cf2c70 100644 --- a/dbms/src/Common/BackgroundSchedulePool.h +++ b/dbms/src/Common/BackgroundSchedulePool.h @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { @@ -135,6 +136,9 @@ private: std::thread delayed_thread; /// Tasks ordered by scheduled time. DelayedTasks delayed_tasks; + + /// Thread group used for profiling purposes + ThreadGroupStatusPtr thread_group; }; using BackgroundSchedulePoolPtr = std::shared_ptr; diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 7b07bd44a7e..dcc6cc7bdfa 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -35,34 +35,19 @@ void CurrentThread::initializeQuery() getCurrentThreadImpl()->initializeQuery(); } -void CurrentThread::attachQuery(QueryStatus * parent_process) +void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group) { - ThreadStatusPtr thread = getCurrentThreadImpl(); - - if (!parent_process) - thread->attachQuery(nullptr, nullptr, nullptr, CurrentThread::getInternalTextLogsQueue()); - else - { - thread->attachQuery( - parent_process, &parent_process->performance_counters, &parent_process->memory_tracker, - CurrentThread::getInternalTextLogsQueue()); - } + getCurrentThreadImpl()->attachQuery(thread_group, true); } - -void CurrentThread::attachQueryFromSiblingThread(const ThreadStatusPtr & sibling_thread) +void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group) { - attachQueryFromSiblingThreadImpl(sibling_thread, true); -} - -void CurrentThread::attachQueryFromSiblingThreadIfDetached(const ThreadStatusPtr & sibling_thread) -{ - attachQueryFromSiblingThreadImpl(sibling_thread, false); + getCurrentThreadImpl()->attachQuery(thread_group, false); } void CurrentThread::updatePerformanceCounters() { - getCurrentThreadImpl()->updatePerformanceCountersImpl(); + getCurrentThreadImpl()->updatePerformanceCounters(); } ThreadStatusPtr CurrentThread::get() @@ -100,49 +85,9 @@ void CurrentThread::updateProgressOut(const Progress & value) current_thread->progress_out.incrementPiecewiseAtomically(value); } -void CurrentThread::attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thread, bool check_detached) +void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr & logs_queue) { - if (sibling_thread == nullptr) - throw Exception("Sibling thread was not initialized", ErrorCodes::LOGICAL_ERROR); - - ThreadStatusPtr thread = getCurrentThreadImpl(); - - if (sibling_thread->getCurrentState() == ThreadStatus::ThreadState::QueryInitializing) - { - LOG_WARNING(thread->log, "An attempt to \'fork\' from initializing thread detected." - << " Performance statistics for this thread will be inaccurate"); - } - - QueryStatus * parent_query; - ProfileEvents::Counters * parent_counters; - MemoryTracker * parent_memory_tracker; - InternalTextLogsQueueWeakPtr logs_queue_ptr; - { - /// NOTE: It is almost the only place where ThreadStatus::mutex is required - /// In other cases ThreadStatus must be accessed only from the current_thread - std::lock_guard lock(sibling_thread->mutex); - - parent_query = sibling_thread->parent_query; - if (parent_query) - { - parent_counters = &parent_query->performance_counters; - parent_memory_tracker = &parent_query->memory_tracker; - } - else - { - /// Fallback - parent_counters = sibling_thread->performance_counters.getParent(); - parent_memory_tracker = sibling_thread->memory_tracker.getParent(); - } - logs_queue_ptr = sibling_thread->logs_queue_ptr; - } - - thread->attachQuery(parent_query, parent_counters, parent_memory_tracker, logs_queue_ptr, check_detached); -} - -void CurrentThread::attachSystemLogsQueue(const std::shared_ptr & logs_queue) -{ - getCurrentThreadImpl()->attachSystemLogsQueue(logs_queue); + getCurrentThreadImpl()->attachInternalTextLogsQueue(logs_queue); } std::shared_ptr CurrentThread::getInternalTextLogsQueue() @@ -164,13 +109,22 @@ std::string CurrentThread::getCurrentQueryID() if (!current_thread || current_thread.use_count() <= 0) return {}; - if (current_thread->parent_query) - return current_thread->parent_query->client_info.current_query_id; + return current_thread->getQueryID(); +} - if (current_thread->query_context) - return current_thread->query_context->getClientInfo().current_query_id; +ThreadGroupStatusPtr CurrentThread::getGroup() +{ + return getCurrentThreadImpl()->getThreadGroup(); +} - return {}; +void CurrentThread::attachQueryContext(Context & query_context) +{ + return getCurrentThreadImpl()->attachQueryContext(query_context); +} + +void CurrentThread::finalizePerformanceCounters() +{ + getCurrentThreadImpl()->finalizePerformanceCounters(); } } diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index 7fc9a03a7e0..1c7deba594f 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -14,11 +14,14 @@ class MemoryTracker; namespace DB { +class Context; class QueryStatus; class ThreadStatus; struct Progress; using ThreadStatusPtr = std::shared_ptr; class InternalTextLogsQueue; +class ThreadGroupStatus; +using ThreadGroupStatusPtr = std::shared_ptr; class CurrentThread @@ -26,25 +29,28 @@ class CurrentThread public: static ThreadStatusPtr get(); + static ThreadGroupStatusPtr getGroup(); - /// Call when thread accepted connection (but haven't called executeQuery()) - /// Currently it is used only for debugging + /// Call from master thread as soon as possible (e.g. when thread accepted connection) static void initializeQuery(); /// A logs queue used by TCPHandler to pass logs to a client - static void attachSystemLogsQueue(const std::shared_ptr & logs_queue); + static void attachInternalTextLogsQueue(const std::shared_ptr & logs_queue); static std::shared_ptr getInternalTextLogsQueue(); + /// Sets query_context for current thread group + static void attachQueryContext(Context & query_context); + /// You must call one of these methods when create a child thread: - /// Bundles the current thread with a query - static void attachQuery(QueryStatus * parent_process); /// Bundles the current thread with a query bundled to the sibling thread - static void attachQueryFromSiblingThread(const ThreadStatusPtr & sibling_thread); + static void attachTo(const ThreadGroupStatusPtr & thread_group); /// Is useful for a ThreadPool tasks - static void attachQueryFromSiblingThreadIfDetached(const ThreadStatusPtr & sibling_thread); + static void attachToIfDetached(const ThreadGroupStatusPtr & thread_group); /// Makes system calls to update ProfileEvents derived from rusage and taskstats static void updatePerformanceCounters(); + /// Update ProfileEvents and dumps info to system.query_thread_log + static void finalizePerformanceCounters(); static ProfileEvents::Counters & getProfileEvents(); static MemoryTracker & getMemoryTracker(); @@ -58,9 +64,6 @@ public: /// Non-master threads call this method in destructor automatically static void detachQuery(); static void detachQueryIfNotDetached(); - -private: - static void attachQueryFromSiblingThreadImpl(ThreadStatusPtr sibling_thread, bool check_detached = true); }; } diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index 72f03d17829..7e957ae1ae4 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -18,7 +18,7 @@ namespace DB MemoryTracker::~MemoryTracker() { - if (level != VariableContext::Thread && peak) + if (static_cast(level) < static_cast(VariableContext::Process) && peak) { try { diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 416425833fb..20314b02feb 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -57,7 +57,7 @@ public: { ThreadStatus & thread = *CurrentThread::get(); - LOG_DEBUG(thread.log, "Thread " << thread.thread_number << " exited"); + LOG_TRACE(thread.log, "Thread " << thread.thread_number << " exited"); thread.detachQuery(true, true); } catch (...) @@ -218,18 +218,22 @@ ThreadStatus::~ThreadStatus() = default; void ThreadStatus::initializeQuery() { - if (thread_state != ThreadState::QueryInitializing && thread_state != ThreadState::DetachedFromQuery) - throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); + assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__); - thread_state = ThreadState::QueryInitializing; + thread_group = std::make_shared(); + + performance_counters.setParent(&thread_group->performance_counters); + memory_tracker.setParent(&thread_group->memory_tracker); + thread_group->memory_tracker.setDescription("(for query)"); + + thread_group->master_thread = shared_from_this(); + thread_group->thread_statuses.emplace(thread_number, shared_from_this()); + + initPerformanceCounters(); + thread_state = ThreadState::AttachedToQuery; } -void ThreadStatus::attachQuery( - QueryStatus * parent_query_, - ProfileEvents::Counters * parent_counters, - MemoryTracker * parent_memory_tracker, - const InternalTextLogsQueueWeakPtr & logs_queue_ptr_, - bool check_detached) +void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached) { if (thread_state == ThreadState::AttachedToQuery) { @@ -238,50 +242,68 @@ void ThreadStatus::attachQuery( return; } - if (thread_state != ThreadState::DetachedFromQuery && thread_state != ThreadState::QueryInitializing) - throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); + assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__); + + if (!thread_group_) + throw Exception("Attempt to attach to nullptr thread group", ErrorCodes::LOGICAL_ERROR); + + /// Attach current thread to thread group and copy useful information from it + thread_group = thread_group_; + + performance_counters.setParent(&thread_group->performance_counters); + memory_tracker.setParent(&thread_group->memory_tracker); { - std::lock_guard lock(mutex); - parent_query = parent_query_; - performance_counters.setParent(parent_counters); - memory_tracker.setParent(parent_memory_tracker); - logs_queue_ptr = logs_queue_ptr_; + std::unique_lock lock(thread_group->mutex); + + logs_queue_ptr = thread_group->logs_queue_ptr; + query_context = thread_group->query_context; + + if (!global_context) + global_context = thread_group->global_context; + + if (!thread_group->thread_statuses.emplace(thread_number, shared_from_this()).second) + throw Exception("Thread " + std::to_string(thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); } + initPerformanceCounters(); + thread_state = ThreadState::AttachedToQuery; +} + +void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) +{ + if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery) + { + thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; + return; + } + + assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__); + finalizePerformanceCounters(); + + /// For better logging ({query_id} will be shown here) + if (thread_group && thread_group.use_count() == 1) + thread_group->memory_tracker.logPeakMemoryUsage(); + + /// Detach from thread group + performance_counters.setParent(&ProfileEvents::global_counters); + memory_tracker.setParent(nullptr); + query_context = nullptr; + thread_group.reset(); + + thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; +} + +void ThreadStatus::initPerformanceCounters() +{ + performance_counters_finalized = false; + /// Clear stats from previous query if a new query is started /// TODO: make separate query_thread_performance_counters and thread_performance_counters performance_counters.resetCounters(); memory_tracker.resetCounters(); memory_tracker.setDescription("(for thread)"); - /// Try extract as many information as possible from ProcessList - if (parent_query) - { - /// Attach current thread to list of query threads - { - std::unique_lock lock(parent_query->threads_mutex); - - if (parent_query->thread_statuses.empty()) - parent_query->master_thread = shared_from_this(); - - if (!parent_query->thread_statuses.emplace(thread_number, shared_from_this()).second) - throw Exception("Thread " + std::to_string(thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); - } - - query_context = parent_query->tryGetQueryContext(); - if (query_context) - { - log_to_query_thread_log = query_context->getSettingsRef().log_query_threads.value != 0; - log_profile_events = query_context->getSettingsRef().log_profile_events.value != 0; - - if (!getGlobalContext()) - global_context = &query_context->getGlobalContext(); - } - } - - thread_state = ThreadState::AttachedToQuery; - query_start_time_nanoseconds = getCurrentTimeNanoseconds(); query_start_time = time(nullptr); ++queries_started; @@ -292,47 +314,7 @@ void ThreadStatus::attachQuery( *last_taskstats = TasksStatsCounters::current(); } -void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) -{ - if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery) - { - thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; - return; - } - - if (thread_state != ThreadState::AttachedToQuery && thread_state != ThreadState::QueryInitializing) - throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); - - updatePerformanceCountersImpl(); - - try - { - if (log_to_query_thread_log) - if (auto global_context = getGlobalContext()) - if (auto thread_log = global_context->getQueryThreadLog()) - logToQueryThreadLog(*thread_log); - } - catch (...) - { - tryLogCurrentException(log); - } - - { - std::lock_guard lock(mutex); - - /// Detach from parent - performance_counters.setParent(&ProfileEvents::global_counters); - memory_tracker.setParent(nullptr); - query_context = nullptr; - } - - thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; - log_to_query_thread_log = true; - log_profile_events = true; -} - - -void ThreadStatus::updatePerformanceCountersImpl() +void ThreadStatus::updatePerformanceCounters() { try { @@ -346,6 +328,28 @@ void ThreadStatus::updatePerformanceCountersImpl() } } +void ThreadStatus::finalizePerformanceCounters() +{ + if (performance_counters_finalized) + return; + + performance_counters_finalized = true; + updatePerformanceCounters(); + + try + { + bool log_to_query_thread_log = global_context && query_context && query_context->getSettingsRef().log_query_threads.value != 0; + if (log_to_query_thread_log) + if (auto thread_log = global_context->getQueryThreadLog()) + logToQueryThreadLog(*thread_log); + } + catch (...) + { + tryLogCurrentException(log); + } +} + + void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) { QueryThreadLogElement elem; @@ -364,40 +368,83 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) elem.thread_number = thread_number; elem.os_thread_id = os_thread_id; - if (parent_query) + if (thread_group) { { - std::shared_lock threads_mutex(parent_query->threads_mutex); + std::shared_lock lock(thread_group->mutex); - if (parent_query->master_thread) + if (thread_group->master_thread) { - elem.master_thread_number = parent_query->master_thread->thread_number; - elem.master_os_thread_id = parent_query->master_thread->os_thread_id; + elem.master_thread_number = thread_group->master_thread->thread_number; + elem.master_os_thread_id = thread_group->master_thread->os_thread_id; } - } - elem.query = parent_query->query; - elem.client_info = parent_query->getClientInfo(); + elem.query = thread_group->query; + } } - if (log_profile_events) + if (query_context) { - /// NOTE: Here we are in the same thread, so we can make memcpy() - elem.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); + elem.client_info = query_context->getClientInfo(); + + if (query_context->getSettingsRef().log_profile_events.value != 0) + { + /// NOTE: Here we are in the same thread, so we can make memcpy() + elem.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); + } } thread_log.add(elem); } -void ThreadStatus::clean() + +void ThreadStatus::assertState(const std::initializer_list & permitted_states, const char * description) { + for (auto permitted_state : permitted_states) { - std::lock_guard lock(mutex); - parent_query = nullptr; + if (getCurrentState() == permitted_state) + return; } - if (thread_state != ThreadState::DetachedFromQuery && thread_state != ThreadState::Died) - throw Exception("Unexpected thread state " + std::to_string(getCurrentState()) + __PRETTY_FUNCTION__, ErrorCodes::LOGICAL_ERROR); + std::stringstream ss; + ss << "Unexpected thread state " << getCurrentState(); + if (description) + ss << ": " << description; + throw Exception(ss.str(), ErrorCodes::LOGICAL_ERROR); +} + +void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue) +{ + logs_queue_ptr = logs_queue; + + if (!thread_group) + return; + + std::unique_lock lock(thread_group->mutex); + thread_group->logs_queue_ptr = logs_queue; +} + +void ThreadStatus::attachQueryContext(Context & query_context_) +{ + query_context = &query_context_; + if (!global_context) + global_context = &query_context->getGlobalContext(); + + if (!thread_group) + return; + + std::unique_lock lock(thread_group->mutex); + thread_group->query_context = query_context; + if (!thread_group->global_context) + thread_group->global_context = global_context; +} + +String ThreadStatus::getQueryID() +{ + if (query_context) + return query_context->getClientInfo().current_query_id; + + return {}; } } diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index afe213eeb90..c63d9ec390f 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -3,7 +3,9 @@ #include #include #include +#include #include +#include namespace Poco @@ -31,6 +33,31 @@ using ThreadStatusPtr = std::shared_ptr; extern thread_local ThreadStatusPtr current_thread; +class ThreadGroupStatus +{ +public: + + mutable std::shared_mutex mutex; + + ProfileEvents::Counters performance_counters{VariableContext::Process}; + MemoryTracker memory_tracker{VariableContext::Process}; + + Context * query_context = nullptr; + Context * global_context = nullptr; + + InternalTextLogsQueueWeakPtr logs_queue_ptr; + + /// Key is Poco's thread_id + using QueryThreadStatuses = std::map; + QueryThreadStatuses thread_statuses; + ThreadStatusPtr master_thread; + + String query; +}; + +using ThreadGroupStatusPtr = std::shared_ptr; + + class ThreadStatus : public std::enable_shared_from_this { public: @@ -52,13 +79,14 @@ public: static ThreadStatusPtr create(); - /// Called by master thread when the query finishes - void clean(); + ThreadGroupStatusPtr getThreadGroup() const + { + return thread_group; + } enum ThreadState { - DetachedFromQuery = 0, /// We just created thread or it is background thread - QueryInitializing, /// We accepted a connection, but haven't enqueued a query to ProcessList + DetachedFromQuery = 0, /// We just created thread or it is a background thread AttachedToQuery, /// Thread executes enqueued query Died, /// Thread does not exist }; @@ -68,21 +96,33 @@ public: return thread_state.load(std::memory_order_relaxed); } + String getQueryID(); + + /// Starts new query and create new thread group fro it, current thread becomes master thread of the query + void initializeQuery(); + + /// Attaches slave thread to existing thread group + void attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); + InternalTextLogsQueuePtr getInternalTextLogsQueue() const { return thread_state == Died ? nullptr : logs_queue_ptr.lock(); } - void attachSystemLogsQueue(const InternalTextLogsQueuePtr & logs_queue) - { - std::lock_guard lock(mutex); - logs_queue_ptr = logs_queue; - } + void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue); - Context * getGlobalContext() - { - return global_context.load(std::memory_order_relaxed); - } + /// Sets query context for current thread and its thread group + /// NOTE: query_context have to be alive until detachQuery() is called + void attachQueryContext(Context & query_context); + + /// Update several ProfileEvents counters + void updatePerformanceCounters(); + + /// Update ProfileEvents and dumps info to system.query_thread_log + void finalizePerformanceCounters(); + + /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped + void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false); ~ThreadStatus(); @@ -90,39 +130,27 @@ protected: ThreadStatus(); - void initializeQuery(); - - void attachQuery( - QueryStatus * parent_query_, - ProfileEvents::Counters * parent_counters, - MemoryTracker * parent_memory_tracker, - const InternalTextLogsQueueWeakPtr & logs_queue_ptr_, - bool check_detached = true); - - void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false); + void initPerformanceCounters(); void logToQueryThreadLog(QueryThreadLog & thread_log); - void updatePerformanceCountersImpl(); + void assertState(const std::initializer_list & permitted_states, const char * description = nullptr); + + ThreadGroupStatusPtr thread_group; std::atomic thread_state{ThreadState::DetachedFromQuery}; - mutable std::mutex mutex; - QueryStatus * parent_query = nullptr; - /// Is set once - std::atomic global_context{nullptr}; + Context * global_context = nullptr; /// Use it only from current thread Context * query_context = nullptr; /// A logs queue used by TCPHandler to pass logs to a client InternalTextLogsQueueWeakPtr logs_queue_ptr; + bool performance_counters_finalized = false; UInt64 query_start_time_nanoseconds = 0; time_t query_start_time = 0; - - bool log_to_query_thread_log = true; - bool log_profile_events = true; size_t queries_started = 0; Poco::Logger * log = nullptr; diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp index dda2378ef88..1e957506aa4 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp @@ -35,7 +35,7 @@ void AsynchronousBlockInputStream::next() { ready.reset(); - pool.schedule([this, main_thread=CurrentThread::get()] () + pool.schedule([this, thread_group=CurrentThread::getGroup()] () { CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; @@ -43,7 +43,7 @@ void AsynchronousBlockInputStream::next() { if (first) setThreadName("AsyncBlockInput"); - CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + CurrentThread::attachToIfDetached(thread_group); } catch (...) { diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index 0c42edacef8..1a891525750 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -176,10 +176,10 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start() { auto & child = children[i]; - auto main_thread = CurrentThread::get(); - reading_pool->schedule([&child, main_thread] + auto thread_group = CurrentThread::getGroup(); + reading_pool->schedule([&child, thread_group] { - CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + CurrentThread::attachToIfDetached(thread_group); setThreadName("MergeAggReadThr"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; child->readPrefix(); @@ -197,7 +197,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start() */ for (size_t i = 0; i < merging_threads; ++i) - pool.schedule([this] () { mergeThread(current_thread); } ); + pool.schedule([this, thread_group=CurrentThread::getGroup()] () { mergeThread(thread_group); } ); } } @@ -293,15 +293,16 @@ void MergingAggregatedMemoryEfficientBlockInputStream::finalize() } -void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadStatusPtr main_thread) +void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadGroupStatusPtr thread_group) { - if (main_thread) - CurrentThread::attachQueryFromSiblingThread(main_thread); - setThreadName("MergeAggMergThr"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; try { + if (thread_group) + CurrentThread::attachTo(thread_group); + setThreadName("MergeAggMergThr"); + while (!parallel_merge_data->finish) { /** Receiving next blocks is processing by one thread pool, and merge is in another. @@ -481,10 +482,10 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate { if (need_that_input(input)) { - auto main_thread = current_thread; - reading_pool->schedule([&input, &read_from_input, main_thread] + auto thread_group = CurrentThread::getGroup(); + reading_pool->schedule([&input, &read_from_input, thread_group] { - CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + CurrentThread::attachToIfDetached(thread_group); setThreadName("MergeAggReadThr"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; read_from_input(input); diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index 5599f8e1120..bdabd8cc1f8 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include @@ -152,7 +152,7 @@ private: std::unique_ptr parallel_merge_data; - void mergeThread(ThreadStatusPtr main_thread); + void mergeThread(ThreadGroupStatusPtr main_thread); void finalize(); }; diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index 1af47062b13..a1c4e2ac480 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -106,9 +106,9 @@ public: { active_threads = max_threads; threads.reserve(max_threads); - auto main_thread = CurrentThread::get(); + auto thread_group = CurrentThread::getGroup(); for (size_t i = 0; i < max_threads; ++i) - threads.emplace_back([=] () { thread(main_thread, i); } ); + threads.emplace_back([=] () { thread(thread_group, i); } ); } /// Ask all sources to stop earlier than they run out. @@ -176,16 +176,16 @@ private: } } - void thread(ThreadStatusPtr main_thread, size_t thread_num) + void thread(ThreadGroupStatusPtr thread_group, size_t thread_num) { - CurrentThread::attachQueryFromSiblingThread(main_thread); std::exception_ptr exception; - - setThreadName("ParalInputsProc"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; try { + setThreadName("ParalInputsProc"); + CurrentThread::attachTo(thread_group); + while (!finish) { InputData unprepared_input; diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 16095ef7f8d..cfdb377b699 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1274,9 +1274,9 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( bool final, ThreadPool * thread_pool) const { - auto converter = [&](size_t bucket, ThreadStatusPtr main_thread) + auto converter = [&](size_t bucket, ThreadGroupStatusPtr thread_group) { - CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + CurrentThread::attachToIfDetached(thread_group); return convertOneBucketToBlock(data_variants, method, final, bucket); }; @@ -1291,7 +1291,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( if (method.data.impls[bucket].empty()) continue; - tasks[bucket] = std::packaged_task(std::bind(converter, bucket, CurrentThread::get())); + tasks[bucket] = std::packaged_task(std::bind(converter, bucket, CurrentThread::getGroup())); if (thread_pool) thread_pool->schedule([bucket, &tasks] { tasks[bucket](); }); @@ -1721,15 +1721,15 @@ private: return; parallel_merge_data->pool.schedule(std::bind(&MergingAndConvertingBlockInputStream::thread, this, - max_scheduled_bucket_num, CurrentThread::get())); + max_scheduled_bucket_num, CurrentThread::getGroup())); } - void thread(Int32 bucket_num, ThreadStatusPtr main_thread) + void thread(Int32 bucket_num, ThreadGroupStatusPtr thread_group) { try { setThreadName("MergingAggregtd"); - CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + CurrentThread::attachToIfDetached(thread_group); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; /// TODO: add no_more_keys support maybe @@ -2031,9 +2031,9 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV LOG_TRACE(log, "Merging partially aggregated two-level data."); - auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadStatusPtr main_thread) + auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupStatusPtr thread_group) { - CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + CurrentThread::attachToIfDetached(thread_group); for (Block & block : bucket_to_blocks[bucket]) { @@ -2066,7 +2066,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV result.aggregates_pools.push_back(std::make_shared()); Arena * aggregates_pool = result.aggregates_pools.back().get(); - auto task = std::bind(merge_bucket, bucket, aggregates_pool, CurrentThread::get()); + auto task = std::bind(merge_bucket, bucket, aggregates_pool, CurrentThread::getGroup()); if (thread_pool) thread_pool->schedule(task); diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index f96032952d6..30e2624f8b3 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -166,10 +166,27 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as /// Query-level memory tracker is already set in the QueryStatus constructor - /// Attach master thread - CurrentThread::attachQuery(&(*process_it)); - /// 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 + /// Actualize thread group info + { + auto thread_group = CurrentThread::getGroup(); + + std::unique_lock lock(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; + + /// Set memory trackers + thread_group->memory_tracker.setOrRaiseLimit(process_it->max_memory_usage); + thread_group->memory_tracker.setDescription("(for query)"); + + if (process_it->memory_tracker_fault_probability) + thread_group->memory_tracker.setFaultProbability(process_it->memory_tracker_fault_probability); + + /// 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); + } if (!user_process_list.user_throttler) { @@ -195,14 +212,6 @@ ProcessListEntry::~ProcessListEntry() /// Destroy all streams to avoid long lock of ProcessList it->releaseQueryStreams(); - /// Finalize all threads statuses - CurrentThread::detachQueryIfNotDetached(); - { - std::shared_lock lock(it->threads_mutex); - for (auto & elem : it->thread_statuses) - elem.second->clean(); - } - std::lock_guard lock(parent.mutex); String user = it->getClientInfo().current_user; @@ -248,7 +257,7 @@ ProcessListEntry::~ProcessListEntry() /// If there are no more queries for the user, then we will reset memory tracker and network throttler. if (user_process_list.queries.empty()) - user_process_list.reset(); + user_process_list.resetTrackers(); /// This removes memory_tracker for all requests. At this time, no other memory_trackers live. if (parent.processes.size() == 0) @@ -264,20 +273,17 @@ ProcessListEntry::~ProcessListEntry() QueryStatus::QueryStatus( const String & query_, const ClientInfo & client_info_, - size_t max_memory_usage, - double memory_tracker_fault_probability, + size_t max_memory_usage_, + double memory_tracker_fault_probability_, QueryPriorities::Handle && priority_handle_) : query(query_), client_info(client_info_), priority_handle(std::move(priority_handle_)), - num_queries_increment{CurrentMetrics::Query} + num_queries_increment{CurrentMetrics::Query}, + max_memory_usage(max_memory_usage_), + memory_tracker_fault_probability(memory_tracker_fault_probability_) { - memory_tracker.setOrRaiseLimit(max_memory_usage); - memory_tracker.setDescription("(for query)"); - - if (memory_tracker_fault_probability) - memory_tracker.setFaultProbability(memory_tracker_fault_probability); } QueryStatus::~QueryStatus() = default; @@ -330,8 +336,6 @@ bool QueryStatus::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStream void QueryStatus::setUserProcessList(ProcessListForUser * user_process_list_) { user_process_list = user_process_list_; - performance_counters.setParent(&user_process_list->user_performance_counters); - memory_tracker.setParent(&user_process_list->user_memory_tracker); } @@ -403,21 +407,25 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even res.total_rows = progress_in.total_rows; res.written_rows = progress_out.rows; res.written_bytes = progress_out.bytes; - res.memory_usage = memory_tracker.get(); - res.peak_memory_usage = memory_tracker.getPeak(); - if (get_thread_list) + if (thread_group) { - std::shared_lock lock(threads_mutex); - res.thread_numbers.reserve(thread_statuses.size()); + res.memory_usage = thread_group->memory_tracker.get(); + res.peak_memory_usage = thread_group->memory_tracker.getPeak(); - for (auto & thread_status_elem : thread_statuses) - res.thread_numbers.emplace_back(thread_status_elem.second->thread_number); + if (get_thread_list) + { + std::shared_lock lock(thread_group->mutex); + res.thread_numbers.reserve(thread_group->thread_statuses.size()); + + for (auto & thread_status_elem : thread_group->thread_statuses) + res.thread_numbers.emplace_back(thread_status_elem.second->thread_number); + } + + if (get_profile_events) + res.profile_counters = std::make_shared(thread_group->performance_counters.getPartiallyAtomicSnapshot()); } - if (get_profile_events) - res.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); - if (get_settings && query_context) res.query_settings = std::make_shared(query_context->getSettingsRef()); diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index 584143380b4..b2a800c77bd 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -87,6 +87,9 @@ protected: /// Is set once when init Context * query_context = nullptr; + /// Info about all threads involved in query execution + ThreadGroupStatusPtr thread_group; + Stopwatch watch; /// Progress of input stream @@ -96,17 +99,11 @@ protected: QueryPriorities::Handle priority_handle; - ProfileEvents::Counters performance_counters{VariableContext::Process}; - MemoryTracker memory_tracker{VariableContext::Process}; - - mutable std::shared_mutex threads_mutex; - /// Key is Poco's thread_id - using QueryThreadStatuses = std::map; - QueryThreadStatuses thread_statuses; - ThreadStatusPtr master_thread; - CurrentMetrics::Increment num_queries_increment{CurrentMetrics::Query}; + size_t max_memory_usage = 0; + double memory_tracker_fault_probability = 0.0; + std::atomic is_killed { false }; void setUserProcessList(ProcessListForUser * user_process_list_); @@ -218,7 +215,7 @@ struct ProcessListForUser /// Sometimes it is important to reset the MemoryTracker, because it may accumulate skew /// due to the fact that there are cases when memory can be allocated while processing the query, but released later. /// Clears network bandwidth Throttler, so it will not count periods of inactivity. - void reset() + void resetTrackers() { user_memory_tracker.reset(); if (user_throttler) diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index 67e9b3b7850..5bf3dcfd200 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -106,7 +106,7 @@ public: } /// Flush data in the buffer to disk - void flush(); + void flush(bool quiet = false); protected: Context & context; @@ -115,7 +115,6 @@ protected: const String storage_def; StoragePtr table; const size_t flush_interval_milliseconds; - std::mutex flush_mutex; using QueueItem = std::pair; /// First element is shutdown flag for thread. @@ -127,6 +126,7 @@ protected: * than accumulation of large amount of log records (for example, for query log - processing of large amount of queries). */ std::vector data; + std::mutex data_mutex; Logger * log; @@ -192,7 +192,16 @@ void SystemLog::threadFunction() QueueItem element; bool has_element = false; - if (data.empty()) + bool is_empty; + { + std::unique_lock lock(data_mutex); + is_empty = data.empty(); + } + + /// data.size() is increased only in this function + /// TODO: get rid of data and queue duality + + if (is_empty) { queue.pop(element); has_element = true; @@ -214,14 +223,17 @@ void SystemLog::threadFunction() break; } else + { + std::unique_lock lock(data_mutex); data.push_back(element.second); + } } size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000; if (milliseconds_elapsed >= flush_interval_milliseconds) { /// Write data to a table. - flush(); + flush(true); time_after_last_write.restart(); } } @@ -236,12 +248,15 @@ void SystemLog::threadFunction() template -void SystemLog::flush() +void SystemLog::flush(bool quiet) { - std::lock_guard flush_lock(flush_mutex); + std::unique_lock lock(data_mutex); try { + if (quiet && data.empty()) + return; + LOG_TRACE(log, "Flushing system log"); /// We check for existence of the table and create it as needed at every flush. diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 9d5f94f20b6..76ca0eafec0 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -118,6 +118,9 @@ static void onExceptionBeforeStart(const String & query, Context & context, time setExceptionStackTrace(elem); logException(context, elem); + /// Update performance counters before logging to query_log + CurrentThread::finalizePerformanceCounters(); + if (log_queries) if (auto query_log = context.getQueryLog()) query_log->add(elem); @@ -134,6 +137,7 @@ static std::tuple executeQueryImpl( time_t current_time = time(nullptr); context.setQueryContext(context); + CurrentThread::attachQueryContext(context); const Settings & settings = context.getSettingsRef(); @@ -270,7 +274,7 @@ static std::tuple executeQueryImpl( return; /// Update performance counters before logging to query_log - CurrentThread::detachQuery(); + CurrentThread::finalizePerformanceCounters(); QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events); @@ -343,7 +347,7 @@ static std::tuple executeQueryImpl( const Settings & settings = context.getSettingsRef(); /// Update performance counters before logging to query_log - CurrentThread::detachQuery(); + CurrentThread::finalizePerformanceCounters(); if (process_list_elem) { diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index fbd1fadcad4..39e78ffd363 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -193,10 +193,10 @@ void DistributedBlockOutputStream::waitForJobs() ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutputStream::JobReplica & job, const Block & current_block) { - auto main_thread = CurrentThread::get(); - return [this, main_thread, &job, ¤t_block]() + auto thread_group = CurrentThread::getGroup(); + return [this, thread_group, &job, ¤t_block]() { - CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + CurrentThread::attachToIfDetached(thread_group); setThreadName("DistrOutStrProc"); ++job.blocks_started; diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index 45bcb2c3810..af909912028 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -57,6 +58,14 @@ BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_) { LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads"); + /// Put all threads to one thread group + /// The master thread exits immediately + CurrentThread::initializeQuery(); + thread_group = CurrentThread::getGroup(); + LOG_INFO(&Logger::get("BackgroundProcessingPool"), "thread_group " << thread_group.get()); + CurrentThread::detachQuery(); + LOG_INFO(&Logger::get("BackgroundProcessingPool"), "thread_group " << thread_group.get()); + threads.resize(size); for (auto & thread : threads) thread = std::thread([this] { threadFunction(); }); @@ -115,7 +124,10 @@ void BackgroundProcessingPool::threadFunction() { setThreadName("BackgrProcPool"); - CurrentThread::attachQuery(nullptr); + /// Put all threads to one thread pool + CurrentThread::attachTo(thread_group); + SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); }); + CurrentThread::getMemoryTracker().setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool); pcg64 rng(randomSeed()); diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h index 0495b5e8c9d..2b61413c5a8 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h @@ -12,6 +12,8 @@ #include #include #include +#include + namespace DB { @@ -64,6 +66,8 @@ protected: std::atomic shutdown {false}; std::condition_variable wake_event; + /// Thread group used for profiling purposes + ThreadGroupStatusPtr thread_group; void threadFunction(); }; From 358e4ae9bf979de76e2150d28c0a81d426b61a11 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Wed, 20 Jun 2018 18:21:42 +0300 Subject: [PATCH 089/315] Fixed failures in utils, more precise test. [#CLICKHOUSE-2910] --- dbms/programs/local/LocalServer.cpp | 4 ++ dbms/programs/server/HTTPHandler.cpp | 3 +- dbms/src/Common/CurrentThread.cpp | 23 +++++++--- dbms/src/Common/CurrentThread.h | 14 +++++- dbms/src/Common/ProfileEvents.cpp | 2 + dbms/src/Common/RWLockFIFO.cpp | 2 +- dbms/src/Common/ThreadStatus.cpp | 26 +++++++---- dbms/src/Common/ThreadStatus.h | 2 + .../Common/tests/gtest_rw_lock_fifo.cpp.cpp | 2 +- .../AsynchronousBlockInputStream.cpp | 5 ++- .../Interpreters/InternalTextLogsQueue.cpp | 44 +++++++++---------- dbms/src/Interpreters/ProcessList.cpp | 11 ++--- dbms/src/Interpreters/QueryThreadLog.cpp | 2 + dbms/src/Interpreters/QueryThreadLog.h | 1 + dbms/src/Interpreters/executeQuery.cpp | 2 +- .../MergeTree/BackgroundProcessingPool.cpp | 2 - ...4_performance_introspection_and_logging.sh | 4 +- libs/libdaemon/src/OwnSplitChannel.cpp | 2 + 18 files changed, 95 insertions(+), 56 deletions(-) diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index 3d90595c98c..fff89dcd7d5 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -269,6 +270,9 @@ void LocalServer::processQueries() context->setCurrentQueryId(""); applyCmdSettings(*context); + /// Use the same query_id (and thread group) for all queries + CurrentThread::QueryScope query_scope_holder(*context); + bool echo_query = config().hasOption("echo") || config().hasOption("verbose"); std::exception_ptr exception; diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index ce5b1416720..a0604a69f30 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -212,10 +212,9 @@ void HTTPHandler::processQuery( Context context = server.context(); context.setGlobalContext(server.context()); - CurrentThread::initializeQuery(); /// It will forcibly detach query even if unexpected error ocurred and detachQuery() was not called /// Normal detaching is happen in BlockIO callbacks - SCOPE_EXIT({CurrentThread::detachQueryIfNotDetached();}); + CurrentThread::QueryScope query_scope_holder(context); LOG_TRACE(log, "Request URI: " << request.getURI()); diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index dcc6cc7bdfa..62f644bc161 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -57,7 +57,7 @@ ThreadStatusPtr CurrentThread::get() void CurrentThread::detachQuery() { - getCurrentThreadImpl()->detachQuery(); + getCurrentThreadImpl()->detachQuery(false); } void CurrentThread::detachQueryIfNotDetached() @@ -112,11 +112,6 @@ std::string CurrentThread::getCurrentQueryID() return current_thread->getQueryID(); } -ThreadGroupStatusPtr CurrentThread::getGroup() -{ - return getCurrentThreadImpl()->getThreadGroup(); -} - void CurrentThread::attachQueryContext(Context & query_context) { return getCurrentThreadImpl()->attachQueryContext(query_context); @@ -127,4 +122,20 @@ void CurrentThread::finalizePerformanceCounters() getCurrentThreadImpl()->finalizePerformanceCounters(); } +ThreadGroupStatusPtr CurrentThread::getGroup() +{ + return getCurrentThreadImpl()->getThreadGroup(); +} + +CurrentThread::QueryScope::~QueryScope() +{ + try + { + CurrentThread::detachQueryIfNotDetached(); + } + catch (...) + { + tryLogCurrentException("CurrentThread", __PRETTY_FUNCTION__); + } +} } diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index 1c7deba594f..de5a2d1f7b4 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -64,8 +64,20 @@ public: /// Non-master threads call this method in destructor automatically static void detachQuery(); static void detachQueryIfNotDetached(); + + + /// Initializes query with current thread as master thread in constructor, and detaches it in desstructor + struct QueryScope + { + explicit QueryScope(Context & query_context) + { + CurrentThread::initializeQuery(); + CurrentThread::attachQueryContext(query_context); + } + + ~QueryScope(); + }; }; } - diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 26fbe0003e3..b4f9883a93f 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -161,6 +161,8 @@ M(InvoluntaryContextSwitches) \ \ M(OSIOWaitMicroseconds) \ + M(OSCPUWaitMicroseconds) \ + M(OSCPUVirtualTimeMicroseconds) \ M(OSReadBytes) \ M(OSWriteBytes) \ M(OSReadChars) \ diff --git a/dbms/src/Common/RWLockFIFO.cpp b/dbms/src/Common/RWLockFIFO.cpp index 51a2f756475..a1211a0bb9d 100644 --- a/dbms/src/Common/RWLockFIFO.cpp +++ b/dbms/src/Common/RWLockFIFO.cpp @@ -86,7 +86,7 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C handler_ptr->it_client->info += "; " + client.info; - return handler_ptr; + return handler_ptr; } if (type == Type::Write || queue.empty() || queue.back().type == Type::Write) diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 20314b02feb..8cc7b8d1797 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -26,10 +26,12 @@ namespace ProfileEvents extern const Event InvoluntaryContextSwitches; extern const Event OSIOWaitMicroseconds; - extern const Event OSReadBytes; - extern const Event OSWriteBytes; + extern const Event OSCPUWaitMicroseconds; + extern const Event OSCPUVirtualTimeMicroseconds; extern const Event OSReadChars; extern const Event OSWriteChars; + extern const Event OSReadBytes; + extern const Event OSWriteBytes; } @@ -99,8 +101,6 @@ struct RusageCounters UInt64 soft_page_faults = 0; UInt64 hard_page_faults = 0; - UInt64 voluntary_context_switches = 0; - UInt64 involuntary_context_switches = 0; RusageCounters() = default; RusageCounters(const ::rusage & rusage_, UInt64 real_time_) @@ -116,8 +116,6 @@ struct RusageCounters soft_page_faults = static_cast(rusage.ru_minflt); hard_page_faults = static_cast(rusage.ru_majflt); - voluntary_context_switches = static_cast(rusage.ru_nvcsw); - involuntary_context_switches = static_cast(rusage.ru_nivcsw); } static RusageCounters zeros(UInt64 real_time_ = getCurrentTimeNanoseconds()) @@ -169,12 +167,21 @@ struct TasksStatsCounters static void incrementProfileEvents(const TasksStatsCounters & prev, const TasksStatsCounters & curr, ProfileEvents::Counters & profile_events) { + profile_events.increment(ProfileEvents::OSCPUWaitMicroseconds, + safeDiff(prev.stat.cpu_delay_total, curr.stat.cpu_delay_total) / 1000U); profile_events.increment(ProfileEvents::OSIOWaitMicroseconds, safeDiff(prev.stat.blkio_delay_total, curr.stat.blkio_delay_total) / 1000U); - profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.stat.read_bytes, curr.stat.read_bytes)); - profile_events.increment(ProfileEvents::OSWriteBytes, safeDiff(prev.stat.write_bytes, curr.stat.write_bytes)); + profile_events.increment(ProfileEvents::OSCPUVirtualTimeMicroseconds, + safeDiff(prev.stat.cpu_run_virtual_total, curr.stat.cpu_run_virtual_total) / 1000U); + + /// Too old struct version, do not read new fields + if (curr.stat.version < TASKSTATS_VERSION) + return; + profile_events.increment(ProfileEvents::OSReadChars, safeDiff(prev.stat.read_char, curr.stat.read_char)); profile_events.increment(ProfileEvents::OSWriteChars, safeDiff(prev.stat.write_char, curr.stat.write_char)); + profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.stat.read_bytes, curr.stat.read_bytes)); + profile_events.increment(ProfileEvents::OSWriteBytes, safeDiff(prev.stat.write_bytes, curr.stat.write_bytes)); } static void updateProfileEvents(TasksStatsCounters & last_counters, ProfileEvents::Counters & profile_events) @@ -362,7 +369,8 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) elem.read_bytes = progress_in.bytes.load(std::memory_order_relaxed); elem.written_rows = progress_out.rows.load(std::memory_order_relaxed); elem.written_bytes = progress_out.bytes.load(std::memory_order_relaxed); - elem.memory_usage = memory_tracker.getPeak(); + elem.memory_usage = memory_tracker.get(); + elem.peak_memory_usage = memory_tracker.getPeak(); elem.thread_name = getThreadName(); elem.thread_number = thread_number; diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index c63d9ec390f..09da8378de5 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -50,6 +50,8 @@ public: /// Key is Poco's thread_id using QueryThreadStatuses = std::map; QueryThreadStatuses thread_statuses; + + /// The first thread created this thread group ThreadStatusPtr master_thread; String query; diff --git a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp index 9d1eda9178c..15b01367cac 100644 --- a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp @@ -32,7 +32,7 @@ TEST(Common, RWLockFIFO_1) auto func = [&] (size_t threads, int round) { - for (int i = 0; i < cycles; ++i) + for (int i = 0; i < cycles; ++i) { auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockFIFO::Read : RWLockFIFO::Write; auto sleep_for = std::chrono::duration(std::uniform_int_distribution<>(1, 100)(gen)); diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp index 1e957506aa4..ba31b45bfd2 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp @@ -43,7 +43,10 @@ void AsynchronousBlockInputStream::next() { if (first) setThreadName("AsyncBlockInput"); - CurrentThread::attachToIfDetached(thread_group); + + /// AsynchronousBlockInputStream is used in Client which does not create queries and thread groups + if (thread_group) + CurrentThread::attachToIfDetached(thread_group); } catch (...) { diff --git a/dbms/src/Interpreters/InternalTextLogsQueue.cpp b/dbms/src/Interpreters/InternalTextLogsQueue.cpp index 8f64cfc7094..a27838a4203 100644 --- a/dbms/src/Interpreters/InternalTextLogsQueue.cpp +++ b/dbms/src/Interpreters/InternalTextLogsQueue.cpp @@ -12,21 +12,21 @@ namespace DB { InternalTextLogsQueue::InternalTextLogsQueue() - : ConcurrentBoundedQueue(std::numeric_limits::max()), - max_priority(Poco::Message::Priority::PRIO_INFORMATION) {} + : ConcurrentBoundedQueue(std::numeric_limits::max()), + max_priority(Poco::Message::Priority::PRIO_INFORMATION) {} Block InternalTextLogsQueue::getSampleBlock() { return Block { - {std::make_shared(), "event_time"}, - {std::make_shared(), "event_time_microseconds"}, - {std::make_shared(), "host_name"}, - {std::make_shared(), "query_id"}, - {std::make_shared(), "thread_number"}, - {std::make_shared(), "priority"}, - {std::make_shared(), "source"}, - {std::make_shared(), "text"}, + {std::make_shared(), "event_time"}, + {std::make_shared(), "event_time_microseconds"}, + {std::make_shared(), "host_name"}, + {std::make_shared(), "query_id"}, + {std::make_shared(), "thread_number"}, + {std::make_shared(), "priority"}, + {std::make_shared(), "source"}, + {std::make_shared(), "text"} }; } @@ -38,11 +38,11 @@ MutableColumns InternalTextLogsQueue::getSampleColumns() void InternalTextLogsQueue::pushBlock(Block && log_block) { - static Block sample_block = getSampleBlock(); + static Block sample_block = getSampleBlock(); - if (blocksHaveEqualStructure(sample_block, log_block)) - emplace(log_block.mutateColumns()); - else + if (blocksHaveEqualStructure(sample_block, log_block)) + emplace(log_block.mutateColumns()); + else LOG_WARNING(&Poco::Logger::get("InternalTextLogsQueue"), "Log block have different structure"); } @@ -52,14 +52,14 @@ const char * InternalTextLogsQueue::getPriorityName(int priority) static const char * PRIORITIES [] = { "Unknown", - "Fatal", - "Critical", - "Error", - "Warning", - "Notice", - "Information", - "Debug", - "Trace" + "Fatal", + "Critical", + "Error", + "Warning", + "Notice", + "Information", + "Debug", + "Trace" }; return (priority >= 1 && priority <= 8) ? PRIORITIES[priority] : PRIORITIES[0]; diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 30e2624f8b3..4388bca73e3 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include @@ -164,21 +163,17 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as user_process_list.user_memory_tracker.setOrRaiseLimit(settings.max_memory_usage_for_user); user_process_list.user_memory_tracker.setDescription("(for user)"); - /// Query-level memory tracker is already set in the QueryStatus constructor - /// Actualize thread group info + if (auto thread_group = CurrentThread::getGroup()) { - auto thread_group = CurrentThread::getGroup(); - - std::unique_lock lock(thread_group->mutex); + std::unique_lock 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; - /// Set memory trackers + /// Set query-level memory trackers thread_group->memory_tracker.setOrRaiseLimit(process_it->max_memory_usage); thread_group->memory_tracker.setDescription("(for query)"); - if (process_it->memory_tracker_fault_probability) thread_group->memory_tracker.setFaultProbability(process_it->memory_tracker_fault_probability); diff --git a/dbms/src/Interpreters/QueryThreadLog.cpp b/dbms/src/Interpreters/QueryThreadLog.cpp index 7b9dbe02d83..a00b9f39ec1 100644 --- a/dbms/src/Interpreters/QueryThreadLog.cpp +++ b/dbms/src/Interpreters/QueryThreadLog.cpp @@ -31,6 +31,7 @@ Block QueryThreadLogElement::createBlock() {std::make_shared(), "written_rows"}, {std::make_shared(), "written_bytes"}, {std::make_shared(), "memory_usage"}, + {std::make_shared(), "peak_memory_usage"}, {std::make_shared(), "thread_name"}, {std::make_shared(), "thread_number"}, @@ -81,6 +82,7 @@ void QueryThreadLogElement::appendToBlock(Block & block) const columns[i++]->insert(UInt64(written_bytes)); columns[i++]->insert(Int64(memory_usage)); + columns[i++]->insert(Int64(peak_memory_usage)); columns[i++]->insertData(thread_name.data(), thread_name.size()); columns[i++]->insert(UInt64(thread_number)); diff --git a/dbms/src/Interpreters/QueryThreadLog.h b/dbms/src/Interpreters/QueryThreadLog.h index ce89477126d..36b8dacd1e6 100644 --- a/dbms/src/Interpreters/QueryThreadLog.h +++ b/dbms/src/Interpreters/QueryThreadLog.h @@ -29,6 +29,7 @@ struct QueryThreadLogElement UInt64 written_bytes{}; Int64 memory_usage{}; + Int64 peak_memory_usage{}; String thread_name; UInt32 thread_number{}; diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 76ca0eafec0..3da5732c215 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -351,7 +351,7 @@ static std::tuple executeQueryImpl( if (process_list_elem) { - QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events); + QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events, false); elem.query_duration_ms = info.elapsed_seconds * 1000; diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index af909912028..c7e49f59e7a 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -62,9 +62,7 @@ BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_) /// The master thread exits immediately CurrentThread::initializeQuery(); thread_group = CurrentThread::getGroup(); - LOG_INFO(&Logger::get("BackgroundProcessingPool"), "thread_group " << thread_group.get()); CurrentThread::detachQuery(); - LOG_INFO(&Logger::get("BackgroundProcessingPool"), "thread_group " << thread_group.get()); threads.resize(size); for (auto & thread : threads) diff --git a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh index 3f12c151e3e..6f9462a6de2 100755 --- a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh +++ b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh @@ -39,7 +39,7 @@ $CLICKHOUSE_CLIENT $settings -q " WITH any(query_duration_ms*1000) AS duration, sumIf(PV, PN = 'RealTimeMicroseconds') AS threads_realtime, - sumIf(PV, PN IN ('UserTimeMicroseconds', 'SystemTimeMicroseconds', 'OSIOWaitMicroseconds')) AS threads_time_user_system_io + sumIf(PV, PN IN ('UserTimeMicroseconds', 'SystemTimeMicroseconds', 'OSIOWaitMicroseconds', 'OSCPUWaitMicroseconds')) AS threads_time_user_system_io SELECT --duration, threads_realtime, threads_time_user_system_io, threads_realtime >= 0.99 * duration, @@ -69,7 +69,7 @@ FROM SELECT thread_number, sumIf(PV, PN = 'RealTimeMicroseconds') AS thread_realtime, - sumIf(PV, PN IN ('UserTimeMicroseconds', 'SystemTimeMicroseconds', 'OSIOWaitMicroseconds')) AS thread_time_user_system_io + sumIf(PV, PN IN ('UserTimeMicroseconds', 'SystemTimeMicroseconds', 'OSIOWaitMicroseconds', 'OSCPUWaitMicroseconds')) AS thread_time_user_system_io FROM (SELECT * FROM system.query_thread_log PREWHERE query_id='$query_id' WHERE event_date >= today()-1) ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV diff --git a/libs/libdaemon/src/OwnSplitChannel.cpp b/libs/libdaemon/src/OwnSplitChannel.cpp index 03c9a77a7c8..e6b2cb22013 100644 --- a/libs/libdaemon/src/OwnSplitChannel.cpp +++ b/libs/libdaemon/src/OwnSplitChannel.cpp @@ -51,6 +51,8 @@ void OwnSplitChannel::log(const Poco::Message & msg) logs_queue->emplace(std::move(columns)); } + + /// TODO: Also log to system.internal_text_log table } void OwnSplitChannel::addChannel(Poco::AutoPtr channel) From 1a4157f2a9fcf1a0f28663935db0d4fa4dbc9ef7 Mon Sep 17 00:00:00 2001 From: Vitaliy Lyudvichenko Date: Wed, 20 Jun 2018 20:49:52 +0300 Subject: [PATCH 090/315] clickhouse_common_io does not depend on dbms. [#CLICKHOUSE-2910] Get rid of extra linking dependencies for clickhouse_common_io. [#CLICKHOUSE-2910] --- dbms/programs/copier/ClusterCopier.cpp | 4 + dbms/src/Common/CurrentThread.cpp | 75 +--- dbms/src/Common/CurrentThread.h | 50 +-- dbms/src/Common/MemoryTracker.h | 2 +- dbms/src/Common/ProfileEvents.cpp | 43 --- dbms/src/Common/ProfileEvents.h | 21 +- dbms/src/Common/ThreadProfileEvents.h | 144 ++++++++ dbms/src/Common/ThreadStatus.cpp | 348 +----------------- dbms/src/Common/ThreadStatus.h | 34 +- dbms/src/Interpreters/ProfileEventsExt.cpp | 59 +++ dbms/src/Interpreters/ProfileEventsExt.h | 12 + dbms/src/Interpreters/QueryLog.cpp | 86 ++--- dbms/src/Interpreters/QueryThreadLog.cpp | 3 +- dbms/src/Interpreters/ThreadStatusExt.cpp | 256 +++++++++++++ .../System/StorageSystemProcesses.cpp | 10 +- ...introspection_and_logging_shard.reference} | 0 ...rmance_introspection_and_logging_shard.sh} | 0 17 files changed, 604 insertions(+), 543 deletions(-) create mode 100644 dbms/src/Common/ThreadProfileEvents.h create mode 100644 dbms/src/Interpreters/ProfileEventsExt.cpp create mode 100644 dbms/src/Interpreters/ProfileEventsExt.h create mode 100644 dbms/src/Interpreters/ThreadStatusExt.cpp rename dbms/tests/queries/0_stateless/{00634_performance_introspection_and_logging.reference => 00634_performance_introspection_and_logging_shard.reference} (100%) rename dbms/tests/queries/0_stateless/{00634_performance_introspection_and_logging.sh => 00634_performance_introspection_and_logging_shard.sh} (100%) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index a000401f02c..fab1d5af4c5 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -2143,6 +2144,9 @@ void ClusterCopierApp::mainImpl() context->addDatabase(default_database, std::make_shared(default_database)); context->setCurrentDatabase(default_database); + /// Initialize query scope just in case. + CurrentThread::QueryScope query_scope(*context); + auto copier = std::make_unique(task_path, host_id, default_database, *context); copier->setSafeMode(is_safe_mode); copier->setCopyFaultProbability(copy_fault_probability); diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 62f644bc161..38a42cb65c8 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -15,8 +15,12 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +void CurrentThread::updatePerformanceCounters() +{ + get()->updatePerformanceCounters(); +} -static ThreadStatusPtr getCurrentThreadImpl() +ThreadStatusPtr CurrentThread::get() { #ifndef NDEBUG if (!current_thread || current_thread.use_count() <= 0) @@ -29,42 +33,6 @@ static ThreadStatusPtr getCurrentThreadImpl() return current_thread; } - -void CurrentThread::initializeQuery() -{ - getCurrentThreadImpl()->initializeQuery(); -} - -void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group) -{ - getCurrentThreadImpl()->attachQuery(thread_group, true); -} - -void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group) -{ - getCurrentThreadImpl()->attachQuery(thread_group, false); -} - -void CurrentThread::updatePerformanceCounters() -{ - getCurrentThreadImpl()->updatePerformanceCounters(); -} - -ThreadStatusPtr CurrentThread::get() -{ - return getCurrentThreadImpl(); -} - -void CurrentThread::detachQuery() -{ - getCurrentThreadImpl()->detachQuery(false); -} - -void CurrentThread::detachQueryIfNotDetached() -{ - getCurrentThreadImpl()->detachQuery(true); -} - ProfileEvents::Counters & CurrentThread::getProfileEvents() { return current_thread->performance_counters; @@ -87,7 +55,7 @@ void CurrentThread::updateProgressOut(const Progress & value) void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr & logs_queue) { - getCurrentThreadImpl()->attachInternalTextLogsQueue(logs_queue); + get()->attachInternalTextLogsQueue(logs_queue); } std::shared_ptr CurrentThread::getInternalTextLogsQueue() @@ -104,38 +72,9 @@ std::shared_ptr CurrentThread::getInternalTextLogsQueue() return current_thread->getInternalTextLogsQueue(); } -std::string CurrentThread::getCurrentQueryID() -{ - if (!current_thread || current_thread.use_count() <= 0) - return {}; - - return current_thread->getQueryID(); -} - -void CurrentThread::attachQueryContext(Context & query_context) -{ - return getCurrentThreadImpl()->attachQueryContext(query_context); -} - -void CurrentThread::finalizePerformanceCounters() -{ - getCurrentThreadImpl()->finalizePerformanceCounters(); -} - ThreadGroupStatusPtr CurrentThread::getGroup() { - return getCurrentThreadImpl()->getThreadGroup(); + return get()->getThreadGroup(); } -CurrentThread::QueryScope::~QueryScope() -{ - try - { - CurrentThread::detachQueryIfNotDetached(); - } - catch (...) - { - tryLogCurrentException("CurrentThread", __PRETTY_FUNCTION__); - } -} } diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index de5a2d1f7b4..0443a01be16 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -28,53 +28,53 @@ class CurrentThread { public: + /// Handler to current thread static ThreadStatusPtr get(); + /// Group to which belongs current thread static ThreadGroupStatusPtr getGroup(); - /// Call from master thread as soon as possible (e.g. when thread accepted connection) - static void initializeQuery(); - /// A logs queue used by TCPHandler to pass logs to a client static void attachInternalTextLogsQueue(const std::shared_ptr & logs_queue); static std::shared_ptr getInternalTextLogsQueue(); - /// Sets query_context for current thread group - static void attachQueryContext(Context & query_context); - - /// You must call one of these methods when create a child thread: - /// Bundles the current thread with a query bundled to the sibling thread - static void attachTo(const ThreadGroupStatusPtr & thread_group); - /// Is useful for a ThreadPool tasks - static void attachToIfDetached(const ThreadGroupStatusPtr & thread_group); - - /// Makes system calls to update ProfileEvents derived from rusage and taskstats + /// Makes system calls to update ProfileEvents that contain info from rusage and taskstats static void updatePerformanceCounters(); - /// Update ProfileEvents and dumps info to system.query_thread_log - static void finalizePerformanceCounters(); static ProfileEvents::Counters & getProfileEvents(); static MemoryTracker & getMemoryTracker(); - /// Returns a non-empty string if the thread is attached to a query - static std::string getCurrentQueryID(); - + /// Update read and write rows (bytes) statistics (used in system.query_thread_log) static void updateProgressIn(const Progress & value); static void updateProgressOut(const Progress & value); + /// Query management: + + /// Call from master thread as soon as possible (e.g. when thread accepted connection) + static void initializeQuery(); + + /// Sets query_context for current thread group + static void attachQueryContext(Context & query_context); + + /// You must call one of these methods when create a query child thread: + /// Add current thread to a group associated with thr thread group + static void attachTo(const ThreadGroupStatusPtr & thread_group); + /// Is useful for a ThreadPool tasks + static void attachToIfDetached(const ThreadGroupStatusPtr & thread_group); + + /// Update ProfileEvents and dumps info to system.query_thread_log + static void finalizePerformanceCounters(); + + /// Returns a non-empty string if the thread is attached to a query + static std::string getCurrentQueryID(); + /// Non-master threads call this method in destructor automatically static void detachQuery(); static void detachQueryIfNotDetached(); - /// Initializes query with current thread as master thread in constructor, and detaches it in desstructor struct QueryScope { - explicit QueryScope(Context & query_context) - { - CurrentThread::initializeQuery(); - CurrentThread::attachQueryContext(query_context); - } - + explicit QueryScope(Context & query_context); ~QueryScope(); }; }; diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index bf6a0a42bc0..68c145393fe 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -1,10 +1,10 @@ #pragma once -#include "VariableContext.h" #include #include #include #include +#include namespace CurrentMetrics diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index b4f9883a93f..8a1f98bbdf0 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -237,49 +237,6 @@ void increment(Event event, Count amount) DB::CurrentThread::getProfileEvents().increment(event, amount); } -void Counters::dumpToArrayColumns(DB::IColumn * column_names_, DB::IColumn * column_values_, bool nonzero_only) -{ - /// Convert ptr and make simple check - auto column_names = (column_names_) ? &typeid_cast(*column_names_) : nullptr; - auto column_values = (column_values_) ? &typeid_cast(*column_values_) : nullptr; - - size_t size = 0; - - for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) - { - UInt64 value = counters[event].load(std::memory_order_relaxed); - - if (nonzero_only && 0 == value) - continue; - - ++size; - - if (column_names) - { - const char * desc = ProfileEvents::getDescription(event); - column_names->getData().insertData(desc, strlen(desc)); - } - - if (column_values) - column_values->getData().insert(value); - } - - if (column_names) - { - auto & offsets = column_names->getOffsets(); - offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size); - } - - /// Nested columns case - bool the_same_offsets = column_names && column_values && column_names->getOffsetsPtr() == column_values->getOffsetsPtr(); - - if (column_values && !the_same_offsets) - { - auto & offsets = column_values->getOffsets(); - offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size); - } -} - } #undef APPLY_FOR_EVENTS diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index ef18ef5531a..7e79a500867 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -1,16 +1,9 @@ #pragma once -#include "VariableContext.h" -#include +#include #include #include - -namespace DB -{ - -class IColumn; - -} +#include /** Implements global counters for various events happening in the application * - for high level profiling. @@ -46,7 +39,12 @@ namespace ProfileEvents Counters(Counter * allocated_counters) : counters(allocated_counters), parent(nullptr), level(VariableContext::Global) {} - inline Counter & operator[] (Event event) + Counter & operator[] (Event event) + { + return counters[event]; + } + + const Counter & operator[] (Event event) const { return counters[event]; } @@ -81,9 +79,6 @@ namespace ProfileEvents /// Set all counters to zero void resetCounters(); - /// Dumps profile events to two column Array(String) and Array(UInt64) - void dumpToArrayColumns(DB::IColumn * column_names, DB::IColumn * column_value, bool nonzero_only = true); - static const Event num_counters; }; diff --git a/dbms/src/Common/ThreadProfileEvents.h b/dbms/src/Common/ThreadProfileEvents.h new file mode 100644 index 00000000000..44843abbd94 --- /dev/null +++ b/dbms/src/Common/ThreadProfileEvents.h @@ -0,0 +1,144 @@ +#pragma once +#include +#include + +#include +#include +#include +#include + + +namespace ProfileEvents +{ + extern const Event RealTimeMicroseconds; + extern const Event UserTimeMicroseconds; + extern const Event SystemTimeMicroseconds; + extern const Event SoftPageFaults; + extern const Event HardPageFaults; + extern const Event VoluntaryContextSwitches; + extern const Event InvoluntaryContextSwitches; + + extern const Event OSIOWaitMicroseconds; + extern const Event OSCPUWaitMicroseconds; + extern const Event OSCPUVirtualTimeMicroseconds; + extern const Event OSReadChars; + extern const Event OSWriteChars; + extern const Event OSReadBytes; + extern const Event OSWriteBytes; +} + + +namespace DB +{ + +/// Handles overflow +template +inline TUInt safeDiff(TUInt prev, TUInt curr) +{ + return curr >= prev ? curr - prev : 0; +} + + +inline UInt64 getCurrentTimeNanoseconds(clockid_t clock_type = CLOCK_MONOTONIC) +{ + struct timespec ts; + clock_gettime(clock_type, &ts); + return ts.tv_sec * 1000000000ULL + ts.tv_nsec; +} + + +struct RusageCounters +{ + /// In nanoseconds + UInt64 real_time = 0; + UInt64 user_time = 0; + UInt64 sys_time = 0; + + UInt64 soft_page_faults = 0; + UInt64 hard_page_faults = 0; + + RusageCounters() = default; + RusageCounters(const ::rusage & rusage_, UInt64 real_time_) + { + set(rusage_, real_time_); + } + + void set(const ::rusage & rusage, UInt64 real_time_) + { + real_time = real_time_; + user_time = rusage.ru_utime.tv_sec * 1000000000UL + rusage.ru_utime.tv_usec; + sys_time = rusage.ru_stime.tv_sec * 1000000000UL + rusage.ru_stime.tv_usec; + + soft_page_faults = static_cast(rusage.ru_minflt); + hard_page_faults = static_cast(rusage.ru_majflt); + } + + static RusageCounters zeros(UInt64 real_time_ = getCurrentTimeNanoseconds()) + { + RusageCounters res; + res.real_time = real_time_; + return res; + } + + static RusageCounters current(UInt64 real_time_ = getCurrentTimeNanoseconds()) + { + ::rusage rusage; + ::getrusage(RUSAGE_THREAD, &rusage); + return RusageCounters(rusage, real_time_); + } + + static void incrementProfileEvents(const RusageCounters & prev, const RusageCounters & curr, ProfileEvents::Counters & profile_events) + { + profile_events.increment(ProfileEvents::RealTimeMicroseconds, (curr.real_time - prev.real_time) / 1000U); + profile_events.increment(ProfileEvents::UserTimeMicroseconds, (curr.user_time - prev.user_time) / 1000U); + profile_events.increment(ProfileEvents::SystemTimeMicroseconds, (curr.sys_time - prev.sys_time) / 1000U); + + profile_events.increment(ProfileEvents::SoftPageFaults, curr.soft_page_faults - prev.soft_page_faults); + profile_events.increment(ProfileEvents::HardPageFaults, curr.hard_page_faults - prev.hard_page_faults); + } + + static void updateProfileEvents(RusageCounters & last_counters, ProfileEvents::Counters & profile_events) + { + auto current_counters = current(); + incrementProfileEvents(last_counters, current_counters, profile_events); + last_counters = current_counters; + } +}; + + +struct TasksStatsCounters +{ + ::taskstats stat; + + TasksStatsCounters() = default; + + static TasksStatsCounters current(); + + static void incrementProfileEvents(const TasksStatsCounters & prev, const TasksStatsCounters & curr, ProfileEvents::Counters & profile_events) + { + profile_events.increment(ProfileEvents::OSCPUWaitMicroseconds, + safeDiff(prev.stat.cpu_delay_total, curr.stat.cpu_delay_total) / 1000U); + profile_events.increment(ProfileEvents::OSIOWaitMicroseconds, + safeDiff(prev.stat.blkio_delay_total, curr.stat.blkio_delay_total) / 1000U); + profile_events.increment(ProfileEvents::OSCPUVirtualTimeMicroseconds, + safeDiff(prev.stat.cpu_run_virtual_total, curr.stat.cpu_run_virtual_total) / 1000U); + + /// Too old struct version, do not read new fields + if (curr.stat.version < TASKSTATS_VERSION) + return; + + profile_events.increment(ProfileEvents::OSReadChars, safeDiff(prev.stat.read_char, curr.stat.read_char)); + profile_events.increment(ProfileEvents::OSWriteChars, safeDiff(prev.stat.write_char, curr.stat.write_char)); + profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.stat.read_bytes, curr.stat.read_bytes)); + profile_events.increment(ProfileEvents::OSWriteBytes, safeDiff(prev.stat.write_bytes, curr.stat.write_bytes)); + } + + static void updateProfileEvents(TasksStatsCounters & last_counters, ProfileEvents::Counters & profile_events) + { + auto current_counters = current(); + incrementProfileEvents(last_counters, current_counters, profile_events); + last_counters = current_counters; + } +}; + +} diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 8cc7b8d1797..7d41915cdec 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -1,39 +1,12 @@ #include "ThreadStatus.h" #include -#include #include -#include -#include -#include +#include +#include #include #include -#include -#include -#include -#include - - -namespace ProfileEvents -{ - extern const Event RealTimeMicroseconds; - extern const Event UserTimeMicroseconds; - extern const Event SystemTimeMicroseconds; - extern const Event SoftPageFaults; - extern const Event HardPageFaults; - extern const Event VoluntaryContextSwitches; - extern const Event InvoluntaryContextSwitches; - - extern const Event OSIOWaitMicroseconds; - extern const Event OSCPUWaitMicroseconds; - extern const Event OSCPUVirtualTimeMicroseconds; - extern const Event OSReadChars; - extern const Event OSWriteChars; - extern const Event OSReadBytes; - extern const Event OSWriteBytes; -} - namespace DB { @@ -46,152 +19,11 @@ namespace ErrorCodes } -/// Implicitly finalizes current thread in the destructor -class ThreadStatus::CurrentThreadScope -{ -public: - - CurrentThreadScope() = default; - - ~CurrentThreadScope() - { - try - { - ThreadStatus & thread = *CurrentThread::get(); - - LOG_TRACE(thread.log, "Thread " << thread.thread_number << " exited"); - thread.detachQuery(true, true); - } - catch (...) - { - std::terminate(); - } - } -}; - - -thread_local ThreadStatusPtr current_thread = ThreadStatus::create(); - /// Order of current_thread and current_thread_scope matters -static thread_local ThreadStatus::CurrentThreadScope current_thread_scope; +thread_local ThreadStatusPtr current_thread = ThreadStatus::create(); +thread_local ThreadStatus::CurrentThreadScope current_thread_scope; -/// Handles overflow -template -inline TUInt safeDiff(TUInt prev, TUInt curr) -{ - return curr >= prev ? curr - prev : 0; -} - - -static UInt64 getCurrentTimeNanoseconds(clockid_t clock_type = CLOCK_MONOTONIC) -{ - struct timespec ts; - clock_gettime(clock_type, &ts); - return ts.tv_sec * 1000000000ULL + ts.tv_nsec; -} - - -struct RusageCounters -{ - /// In nanoseconds - UInt64 real_time = 0; - UInt64 user_time = 0; - UInt64 sys_time = 0; - - UInt64 soft_page_faults = 0; - UInt64 hard_page_faults = 0; - - RusageCounters() = default; - RusageCounters(const ::rusage & rusage_, UInt64 real_time_) - { - set(rusage_, real_time_); - } - - void set(const ::rusage & rusage, UInt64 real_time_) - { - real_time = real_time_; - user_time = rusage.ru_utime.tv_sec * 1000000000UL + rusage.ru_utime.tv_usec; - sys_time = rusage.ru_stime.tv_sec * 1000000000UL + rusage.ru_stime.tv_usec; - - soft_page_faults = static_cast(rusage.ru_minflt); - hard_page_faults = static_cast(rusage.ru_majflt); - } - - static RusageCounters zeros(UInt64 real_time_ = getCurrentTimeNanoseconds()) - { - RusageCounters res; - res.real_time = real_time_; - return res; - } - - static RusageCounters current(UInt64 real_time_ = getCurrentTimeNanoseconds()) - { - ::rusage rusage; - ::getrusage(RUSAGE_THREAD, &rusage); - return RusageCounters(rusage, real_time_); - } - - static void incrementProfileEvents(const RusageCounters & prev, const RusageCounters & curr, ProfileEvents::Counters & profile_events) - { - profile_events.increment(ProfileEvents::RealTimeMicroseconds, (curr.real_time - prev.real_time) / 1000U); - profile_events.increment(ProfileEvents::UserTimeMicroseconds, (curr.user_time - prev.user_time) / 1000U); - profile_events.increment(ProfileEvents::SystemTimeMicroseconds, (curr.sys_time - prev.sys_time) / 1000U); - - profile_events.increment(ProfileEvents::SoftPageFaults, curr.soft_page_faults - prev.soft_page_faults); - profile_events.increment(ProfileEvents::HardPageFaults, curr.hard_page_faults - prev.hard_page_faults); - } - - static void updateProfileEvents(RusageCounters & last_counters, ProfileEvents::Counters & profile_events) - { - auto current_counters = current(); - incrementProfileEvents(last_counters, current_counters, profile_events); - last_counters = current_counters; - } -}; - -struct TasksStatsCounters -{ - ::taskstats stat; - - TasksStatsCounters() = default; - - static TasksStatsCounters zeros() - { - TasksStatsCounters res; - memset(&res.stat, 0, sizeof(stat)); - return res; - } - - static TasksStatsCounters current(); - - static void incrementProfileEvents(const TasksStatsCounters & prev, const TasksStatsCounters & curr, ProfileEvents::Counters & profile_events) - { - profile_events.increment(ProfileEvents::OSCPUWaitMicroseconds, - safeDiff(prev.stat.cpu_delay_total, curr.stat.cpu_delay_total) / 1000U); - profile_events.increment(ProfileEvents::OSIOWaitMicroseconds, - safeDiff(prev.stat.blkio_delay_total, curr.stat.blkio_delay_total) / 1000U); - profile_events.increment(ProfileEvents::OSCPUVirtualTimeMicroseconds, - safeDiff(prev.stat.cpu_run_virtual_total, curr.stat.cpu_run_virtual_total) / 1000U); - - /// Too old struct version, do not read new fields - if (curr.stat.version < TASKSTATS_VERSION) - return; - - profile_events.increment(ProfileEvents::OSReadChars, safeDiff(prev.stat.read_char, curr.stat.read_char)); - profile_events.increment(ProfileEvents::OSWriteChars, safeDiff(prev.stat.write_char, curr.stat.write_char)); - profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.stat.read_bytes, curr.stat.read_bytes)); - profile_events.increment(ProfileEvents::OSWriteBytes, safeDiff(prev.stat.write_bytes, curr.stat.write_bytes)); - } - - static void updateProfileEvents(TasksStatsCounters & last_counters, ProfileEvents::Counters & profile_events) - { - auto current_counters = current(); - incrementProfileEvents(last_counters, current_counters, profile_events); - last_counters = current_counters; - } -}; - TasksStatsCounters TasksStatsCounters::current() { TasksStatsCounters res; @@ -223,84 +55,6 @@ ThreadStatusPtr ThreadStatus::create() ThreadStatus::~ThreadStatus() = default; -void ThreadStatus::initializeQuery() -{ - assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__); - - thread_group = std::make_shared(); - - performance_counters.setParent(&thread_group->performance_counters); - memory_tracker.setParent(&thread_group->memory_tracker); - thread_group->memory_tracker.setDescription("(for query)"); - - thread_group->master_thread = shared_from_this(); - thread_group->thread_statuses.emplace(thread_number, shared_from_this()); - - initPerformanceCounters(); - thread_state = ThreadState::AttachedToQuery; -} - -void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached) -{ - if (thread_state == ThreadState::AttachedToQuery) - { - if (check_detached) - throw Exception("Can't attach query to the thread, it is already attached", ErrorCodes::LOGICAL_ERROR); - return; - } - - assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__); - - if (!thread_group_) - throw Exception("Attempt to attach to nullptr thread group", ErrorCodes::LOGICAL_ERROR); - - /// Attach current thread to thread group and copy useful information from it - thread_group = thread_group_; - - performance_counters.setParent(&thread_group->performance_counters); - memory_tracker.setParent(&thread_group->memory_tracker); - - { - std::unique_lock lock(thread_group->mutex); - - logs_queue_ptr = thread_group->logs_queue_ptr; - query_context = thread_group->query_context; - - if (!global_context) - global_context = thread_group->global_context; - - if (!thread_group->thread_statuses.emplace(thread_number, shared_from_this()).second) - throw Exception("Thread " + std::to_string(thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); - } - - initPerformanceCounters(); - thread_state = ThreadState::AttachedToQuery; -} - -void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) -{ - if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery) - { - thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; - return; - } - - assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__); - finalizePerformanceCounters(); - - /// For better logging ({query_id} will be shown here) - if (thread_group && thread_group.use_count() == 1) - thread_group->memory_tracker.logPeakMemoryUsage(); - - /// Detach from thread group - performance_counters.setParent(&ProfileEvents::global_counters); - memory_tracker.setParent(nullptr); - query_context = nullptr; - thread_group.reset(); - - thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; -} - void ThreadStatus::initPerformanceCounters() { performance_counters_finalized = false; @@ -335,77 +89,6 @@ void ThreadStatus::updatePerformanceCounters() } } -void ThreadStatus::finalizePerformanceCounters() -{ - if (performance_counters_finalized) - return; - - performance_counters_finalized = true; - updatePerformanceCounters(); - - try - { - bool log_to_query_thread_log = global_context && query_context && query_context->getSettingsRef().log_query_threads.value != 0; - if (log_to_query_thread_log) - if (auto thread_log = global_context->getQueryThreadLog()) - logToQueryThreadLog(*thread_log); - } - catch (...) - { - tryLogCurrentException(log); - } -} - - -void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) -{ - QueryThreadLogElement elem; - - elem.event_time = time(nullptr); - elem.query_start_time = query_start_time; - elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U; - - elem.read_rows = progress_in.rows.load(std::memory_order_relaxed); - elem.read_bytes = progress_in.bytes.load(std::memory_order_relaxed); - elem.written_rows = progress_out.rows.load(std::memory_order_relaxed); - elem.written_bytes = progress_out.bytes.load(std::memory_order_relaxed); - elem.memory_usage = memory_tracker.get(); - elem.peak_memory_usage = memory_tracker.getPeak(); - - elem.thread_name = getThreadName(); - elem.thread_number = thread_number; - elem.os_thread_id = os_thread_id; - - if (thread_group) - { - { - std::shared_lock lock(thread_group->mutex); - - if (thread_group->master_thread) - { - elem.master_thread_number = thread_group->master_thread->thread_number; - elem.master_os_thread_id = thread_group->master_thread->os_thread_id; - } - - elem.query = thread_group->query; - } - } - - if (query_context) - { - elem.client_info = query_context->getClientInfo(); - - if (query_context->getSettingsRef().log_profile_events.value != 0) - { - /// NOTE: Here we are in the same thread, so we can make memcpy() - elem.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); - } - } - - thread_log.add(elem); -} - - void ThreadStatus::assertState(const std::initializer_list & permitted_states, const char * description) { for (auto permitted_state : permitted_states) @@ -432,27 +115,4 @@ void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & thread_group->logs_queue_ptr = logs_queue; } -void ThreadStatus::attachQueryContext(Context & query_context_) -{ - query_context = &query_context_; - if (!global_context) - global_context = &query_context->getGlobalContext(); - - if (!thread_group) - return; - - std::unique_lock lock(thread_group->mutex); - thread_group->query_context = query_context; - if (!thread_group->global_context) - thread_group->global_context = global_context; -} - -String ThreadStatus::getQueryID() -{ - if (query_context) - return query_context->getClientInfo().current_query_id; - - return {}; -} - } diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 09da8378de5..3538201dda3 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -20,6 +20,7 @@ namespace DB class Context; class QueryStatus; class ThreadStatus; +using ThreadStatusPtr = std::shared_ptr; class QueryThreadLog; struct TasksStatsCounters; struct RusageCounters; @@ -29,10 +30,6 @@ using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; -using ThreadStatusPtr = std::shared_ptr; -extern thread_local ThreadStatusPtr current_thread; - - class ThreadGroupStatus { public: @@ -167,7 +164,34 @@ protected: bool has_permissions_for_taskstats = false; public: - class CurrentThreadScope; + + /// Implicitly finalizes current thread in the destructor + class CurrentThreadScope + { + public: + void (*deleter)() = nullptr; + + CurrentThreadScope() = default; + ~CurrentThreadScope() + { + try + { + if (deleter) + deleter(); + } + catch (...) + { + std::terminate(); + } + } + }; + +private: + static void defaultThreadDeleter(); }; + +extern thread_local ThreadStatusPtr current_thread; +extern thread_local ThreadStatus::CurrentThreadScope current_thread_scope; + } diff --git a/dbms/src/Interpreters/ProfileEventsExt.cpp b/dbms/src/Interpreters/ProfileEventsExt.cpp new file mode 100644 index 00000000000..8344ac6e40c --- /dev/null +++ b/dbms/src/Interpreters/ProfileEventsExt.cpp @@ -0,0 +1,59 @@ +#include "ProfileEventsExt.h" +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + +/// Put implementation here to avoid extra linking dependencies for clickhouse_common_io +void dumpToArrayColumns(const Counters & counters, DB::IColumn * column_names_, DB::IColumn * column_values_, bool nonzero_only) +{ + /// Convert ptr and make simple check + auto column_names = (column_names_) ? &typeid_cast(*column_names_) : nullptr; + auto column_values = (column_values_) ? &typeid_cast(*column_values_) : nullptr; + + size_t size = 0; + + for (Event event = 0; event < Counters::num_counters; ++event) + { + UInt64 value = counters[event].load(std::memory_order_relaxed); + + if (nonzero_only && 0 == value) + continue; + + ++size; + + if (column_names) + { + const char * desc = ProfileEvents::getDescription(event); + column_names->getData().insertData(desc, strlen(desc)); + } + + if (column_values) + column_values->getData().insert(value); + } + + if (column_names) + { + auto & offsets = column_names->getOffsets(); + offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size); + } + + if (column_values) + { + /// Nested columns case + bool the_same_offsets = column_names && column_names->getOffsetsPtr().get() == column_values->getOffsetsPtr().get(); + if (!the_same_offsets) + { + auto & offsets = column_values->getOffsets(); + offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size); + } + } +} + +} diff --git a/dbms/src/Interpreters/ProfileEventsExt.h b/dbms/src/Interpreters/ProfileEventsExt.h new file mode 100644 index 00000000000..4883505ee64 --- /dev/null +++ b/dbms/src/Interpreters/ProfileEventsExt.h @@ -0,0 +1,12 @@ +#pragma once +#include +#include + + +namespace ProfileEvents +{ + +/// Dumps profile events to two column Array(String) and Array(UInt64) +void dumpToArrayColumns(const Counters & counters, DB::IColumn * column_names, DB::IColumn * column_value, bool nonzero_only = true); + +} diff --git a/dbms/src/Interpreters/QueryLog.cpp b/dbms/src/Interpreters/QueryLog.cpp index eccb8c472f6..568db8b94f6 100644 --- a/dbms/src/Interpreters/QueryLog.cpp +++ b/dbms/src/Interpreters/QueryLog.cpp @@ -1,6 +1,8 @@ +#include #include #include #include +#include #include #include #include @@ -8,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -20,50 +23,50 @@ namespace DB Block QueryLogElement::createBlock() { return - { - {std::make_shared(), "type"}, - {std::make_shared(), "event_date"}, - {std::make_shared(), "event_time"}, - {std::make_shared(), "query_start_time"}, - {std::make_shared(), "query_duration_ms"}, + { + {std::make_shared(), "type"}, + {std::make_shared(), "event_date"}, + {std::make_shared(), "event_time"}, + {std::make_shared(), "query_start_time"}, + {std::make_shared(), "query_duration_ms"}, - {std::make_shared(), "read_rows"}, - {std::make_shared(), "read_bytes"}, - {std::make_shared(), "written_rows"}, - {std::make_shared(), "written_bytes"}, - {std::make_shared(), "result_rows"}, - {std::make_shared(), "result_bytes"}, - {std::make_shared(), "memory_usage"}, + {std::make_shared(), "read_rows"}, + {std::make_shared(), "read_bytes"}, + {std::make_shared(), "written_rows"}, + {std::make_shared(), "written_bytes"}, + {std::make_shared(), "result_rows"}, + {std::make_shared(), "result_bytes"}, + {std::make_shared(), "memory_usage"}, - {std::make_shared(), "query"}, - {std::make_shared(), "exception"}, - {std::make_shared(), "stack_trace"}, + {std::make_shared(), "query"}, + {std::make_shared(), "exception"}, + {std::make_shared(), "stack_trace"}, - {std::make_shared(), "is_initial_query"}, - {std::make_shared(), "user"}, - {std::make_shared(), "query_id"}, - {std::make_shared(16), "address"}, - {std::make_shared(), "port"}, - {std::make_shared(), "initial_user"}, - {std::make_shared(), "initial_query_id"}, - {std::make_shared(16), "initial_address"}, - {std::make_shared(), "initial_port"}, - {std::make_shared(), "interface"}, - {std::make_shared(), "os_user"}, - {std::make_shared(), "client_hostname"}, - {std::make_shared(), "client_name"}, - {std::make_shared(), "client_revision"}, - {std::make_shared(), "http_method"}, - {std::make_shared(), "http_user_agent"}, - {std::make_shared(), "quota_key"}, - {std::make_shared(), "revision"}, + {std::make_shared(), "is_initial_query"}, + {std::make_shared(), "user"}, + {std::make_shared(), "query_id"}, + {std::make_shared(16), "address"}, + {std::make_shared(), "port"}, + {std::make_shared(), "initial_user"}, + {std::make_shared(), "initial_query_id"}, + {std::make_shared(16), "initial_address"}, + {std::make_shared(), "initial_port"}, + {std::make_shared(), "interface"}, + {std::make_shared(), "os_user"}, + {std::make_shared(), "client_hostname"}, + {std::make_shared(), "client_name"}, + {std::make_shared(), "client_revision"}, + {std::make_shared(), "http_method"}, + {std::make_shared(), "http_user_agent"}, + {std::make_shared(), "quota_key"}, + {std::make_shared(), "revision"}, - {std::make_shared(std::make_shared()), "thread_numbers"}, - {std::make_shared(std::make_shared()), "ProfileEvents.Names"}, - {std::make_shared(std::make_shared()), "ProfileEvents.Values"}, - {std::make_shared(std::make_shared()), "Settings.Names"}, - {std::make_shared(std::make_shared()), "Settings.Values"} - }; + {std::make_shared(std::make_shared()), "thread_numbers"}, + {std::make_shared(std::make_shared()), "ProfileEvents.Names"}, + {std::make_shared(std::make_shared()), "ProfileEvents.Values"}, + {std::make_shared(std::make_shared()), "Settings.Names"}, + {std::make_shared(std::make_shared()), "Settings.Values"} + }; } @@ -131,7 +134,7 @@ void QueryLogElement::appendToBlock(Block & block) const { auto column_names = columns[i++].get(); auto column_values = columns[i++].get(); - profile_counters->dumpToArrayColumns(column_names, column_values, true); + ProfileEvents::dumpToArrayColumns(*profile_counters, column_names, column_values, true); } else { @@ -180,5 +183,4 @@ void QueryLogElement::appendClientInfo(const ClientInfo & client_info, MutableCo columns[i++]->insert(client_info.quota_key); } - } diff --git a/dbms/src/Interpreters/QueryThreadLog.cpp b/dbms/src/Interpreters/QueryThreadLog.cpp index a00b9f39ec1..64d80e8017d 100644 --- a/dbms/src/Interpreters/QueryThreadLog.cpp +++ b/dbms/src/Interpreters/QueryThreadLog.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -100,7 +101,7 @@ void QueryThreadLogElement::appendToBlock(Block & block) const { auto column_names = columns[i++].get(); auto column_values = columns[i++].get(); - profile_counters->dumpToArrayColumns(column_names, column_values, true); + dumpToArrayColumns(*profile_counters, column_names, column_values, true); } else { diff --git a/dbms/src/Interpreters/ThreadStatusExt.cpp b/dbms/src/Interpreters/ThreadStatusExt.cpp new file mode 100644 index 00000000000..ffd24a7e906 --- /dev/null +++ b/dbms/src/Interpreters/ThreadStatusExt.cpp @@ -0,0 +1,256 @@ +#include +#include +#include +#include +#include +#include +#include + + +/// Implement some methods of ThreadStatus and CurrentThread here to avoid extra linking dependencies in clickhouse_common_io +namespace DB +{ + +void ThreadStatus::attachQueryContext(Context & query_context_) +{ + query_context = &query_context_; + if (!global_context) + global_context = &query_context->getGlobalContext(); + + if (!thread_group) + return; + + std::unique_lock lock(thread_group->mutex); + thread_group->query_context = query_context; + if (!thread_group->global_context) + thread_group->global_context = global_context; +} + +String ThreadStatus::getQueryID() +{ + if (query_context) + return query_context->getClientInfo().current_query_id; + + return {}; +} + +void ThreadStatus::defaultThreadDeleter() +{ + ThreadStatus & thread = *CurrentThread::get(); + LOG_TRACE(thread.log, "Thread " << thread.thread_number << " exited"); + thread.detachQuery(true, true); +} + +void ThreadStatus::initializeQuery() +{ + assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__); + + thread_group = std::make_shared(); + + performance_counters.setParent(&thread_group->performance_counters); + memory_tracker.setParent(&thread_group->memory_tracker); + thread_group->memory_tracker.setDescription("(for query)"); + + thread_group->master_thread = shared_from_this(); + thread_group->thread_statuses.emplace(thread_number, shared_from_this()); + + initPerformanceCounters(); + thread_state = ThreadState::AttachedToQuery; + current_thread_scope.deleter = ThreadStatus::defaultThreadDeleter; +} + +void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached) +{ + if (thread_state == ThreadState::AttachedToQuery) + { + if (check_detached) + throw Exception("Can't attach query to the thread, it is already attached", ErrorCodes::LOGICAL_ERROR); + return; + } + + assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__); + + if (!thread_group_) + throw Exception("Attempt to attach to nullptr thread group", ErrorCodes::LOGICAL_ERROR); + + /// Attach current thread to thread group and copy useful information from it + thread_group = thread_group_; + + performance_counters.setParent(&thread_group->performance_counters); + memory_tracker.setParent(&thread_group->memory_tracker); + + { + std::unique_lock lock(thread_group->mutex); + + logs_queue_ptr = thread_group->logs_queue_ptr; + query_context = thread_group->query_context; + + if (!global_context) + global_context = thread_group->global_context; + + if (!thread_group->thread_statuses.emplace(thread_number, shared_from_this()).second) + throw Exception("Thread " + std::to_string(thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); + } + + initPerformanceCounters(); + thread_state = ThreadState::AttachedToQuery; + current_thread_scope.deleter = ThreadStatus::defaultThreadDeleter; +} + +void ThreadStatus::finalizePerformanceCounters() +{ + if (performance_counters_finalized) + return; + + performance_counters_finalized = true; + updatePerformanceCounters(); + + try + { + bool log_to_query_thread_log = global_context && query_context && query_context->getSettingsRef().log_query_threads.value != 0; + if (log_to_query_thread_log) + if (auto thread_log = global_context->getQueryThreadLog()) + logToQueryThreadLog(*thread_log); + } + catch (...) + { + tryLogCurrentException(log); + } +} + +void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) +{ + if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery) + { + thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; + return; + } + + assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__); + finalizePerformanceCounters(); + + /// For better logging ({query_id} will be shown here) + if (thread_group && thread_group.use_count() == 1) + thread_group->memory_tracker.logPeakMemoryUsage(); + + /// Detach from thread group + performance_counters.setParent(&ProfileEvents::global_counters); + memory_tracker.setParent(nullptr); + query_context = nullptr; + thread_group.reset(); + + thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery; +} + +void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) +{ + QueryThreadLogElement elem; + + elem.event_time = time(nullptr); + elem.query_start_time = query_start_time; + elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U; + + elem.read_rows = progress_in.rows.load(std::memory_order_relaxed); + elem.read_bytes = progress_in.bytes.load(std::memory_order_relaxed); + elem.written_rows = progress_out.rows.load(std::memory_order_relaxed); + elem.written_bytes = progress_out.bytes.load(std::memory_order_relaxed); + elem.memory_usage = memory_tracker.get(); + elem.peak_memory_usage = memory_tracker.getPeak(); + + elem.thread_name = getThreadName(); + elem.thread_number = thread_number; + elem.os_thread_id = os_thread_id; + + if (thread_group) + { + { + std::shared_lock lock(thread_group->mutex); + + if (thread_group->master_thread) + { + elem.master_thread_number = thread_group->master_thread->thread_number; + elem.master_os_thread_id = thread_group->master_thread->os_thread_id; + } + + elem.query = thread_group->query; + } + } + + if (query_context) + { + elem.client_info = query_context->getClientInfo(); + + if (query_context->getSettingsRef().log_profile_events.value != 0) + { + /// NOTE: Here we are in the same thread, so we can make memcpy() + elem.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); + } + } + + thread_log.add(elem); +} + + +void CurrentThread::initializeQuery() +{ + get()->initializeQuery(); +} + +void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group) +{ + get()->attachQuery(thread_group, true); +} + +void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group) +{ + get()->attachQuery(thread_group, false); +} + +std::string CurrentThread::getCurrentQueryID() +{ + if (!current_thread || current_thread.use_count() <= 0) + return {}; + + return current_thread->getQueryID(); +} + +void CurrentThread::attachQueryContext(Context & query_context) +{ + return get()->attachQueryContext(query_context); +} + +void CurrentThread::finalizePerformanceCounters() +{ + get()->finalizePerformanceCounters(); +} + +void CurrentThread::detachQuery() +{ + get()->detachQuery(false); +} + +void CurrentThread::detachQueryIfNotDetached() +{ + get()->detachQuery(true); +} + + +CurrentThread::QueryScope::QueryScope(Context & query_context) +{ + CurrentThread::initializeQuery(); + CurrentThread::attachQueryContext(query_context); +} + +CurrentThread::QueryScope::~QueryScope() +{ + try + { + CurrentThread::detachQueryIfNotDetached(); + } + catch (...) + { + tryLogCurrentException("CurrentThread", __PRETTY_FUNCTION__); + } +} + +} diff --git a/dbms/src/Storages/System/StorageSystemProcesses.cpp b/dbms/src/Storages/System/StorageSystemProcesses.cpp index bcbc3f147c7..1c9b9f48e0c 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.cpp +++ b/dbms/src/Storages/System/StorageSystemProcesses.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -128,7 +129,14 @@ BlockInputStreams StorageSystemProcesses::read( { IColumn * column_profile_events_names = res_columns[i++].get(); IColumn * column_profile_events_values = res_columns[i++].get(); - process.profile_counters->dumpToArrayColumns(column_profile_events_names, column_profile_events_values, true); + + if (process.profile_counters) + ProfileEvents::dumpToArrayColumns(*process.profile_counters, column_profile_events_names, column_profile_events_values, true); + else + { + column_profile_events_names->insertDefault(); + column_profile_events_values->insertDefault(); + } } { diff --git a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.reference b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging_shard.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.reference rename to dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging_shard.reference diff --git a/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging_shard.sh similarity index 100% rename from dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh rename to dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging_shard.sh From f88584d748b66ed398b2d499419b1d8f3796e632 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 May 2018 19:21:15 +0300 Subject: [PATCH 091/315] Added DeserializeBinaryBulkState which stores reading state for ColumnWithDictionary. --- dbms/src/Columns/ColumnUnique.h | 16 ++-- dbms/src/Columns/ColumnWithDictionary.h | 1 + .../DataStreams/NativeBlockInputStream.cpp | 3 +- dbms/src/DataTypes/DataTypeArray.cpp | 5 +- dbms/src/DataTypes/DataTypeArray.h | 8 +- dbms/src/DataTypes/DataTypeNullable.cpp | 5 +- dbms/src/DataTypes/DataTypeNullable.h | 8 +- dbms/src/DataTypes/DataTypeTuple.cpp | 8 +- dbms/src/DataTypes/DataTypeTuple.h | 19 +++- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 89 +++++++++++++------ dbms/src/DataTypes/DataTypeWithDictionary.h | 11 ++- dbms/src/DataTypes/IDataType.h | 11 ++- .../Storages/MergeTree/MergeTreeReader.cpp | 20 ++--- dbms/src/Storages/MergeTree/MergeTreeReader.h | 3 + dbms/src/Storages/MergeTree/checkDataPart.cpp | 3 +- dbms/src/Storages/StorageLog.cpp | 3 +- dbms/src/Storages/StorageTinyLog.cpp | 3 +- 17 files changed, 155 insertions(+), 61 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 8bc4a2a06a8..bc579829bc3 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -22,8 +22,8 @@ struct StringRefWrapper StringRefWrapper(const ColumnType * column, size_t row) : column(column), row(row) {} StringRefWrapper(StringRef ref) : ref(ref) {} StringRefWrapper(const StringRefWrapper & other) = default; - StringRefWrapper & operator =(int) { column = nullptr; return *this; } - bool operator ==(int) const { return nullptr == column; } + StringRefWrapper & operator =(int) { column = nullptr; ref.data = nullptr; return *this; } + bool operator ==(int) const { return nullptr == column && nullptr == ref.data; } StringRefWrapper() {} operator StringRef() const { return column ? column->getDataAt(row) : ref; } @@ -246,7 +246,7 @@ size_t ColumnUnique::uniqueInsert(const Field & x) if (pos != prev_size) column->popBack(1); - return static_cast(pos); + return pos; } template @@ -262,20 +262,24 @@ size_t ColumnUnique::uniqueInsertFrom(const IColumn & src template size_t ColumnUnique::uniqueInsertData(const char * pos, size_t length) { + if (!index) + buildIndex(); + auto column = getRawColumnPtr(); if (column->getDataAt(getDefaultValueIndex()) == StringRef(pos, length)) return getDefaultValueIndex(); auto size = static_cast(column->size()); + auto iter = index->find(StringRefWrapper(StringRef(pos, length))); - if (!index->has(StringRefWrapper(StringRef(pos, length)))) + if (iter == index->end()) { column->insertData(pos, length); - return static_cast(insertIntoMap(StringRefWrapper(StringRef(pos, length)), size)); + return insertIntoMap(StringRefWrapper(column, size), size); } - return size; + return iter->second; } template diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index dbb89c74ec1..b202f074719 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -81,6 +81,7 @@ public: { if (!typeid_cast(&src)) throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); + auto & src_with_dict = static_cast(src); size_t idx = src_with_dict.getIndexes()->getUInt(n); insertFromFullColumn(*src_with_dict.getUnique()->getNestedColumn(), idx); diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 787ba974546..836fb4d328c 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -60,7 +60,8 @@ NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) { IDataType::InputStreamGetter input_stream_getter = [&] (const IDataType::SubstreamPath &) { return &istr; }; - type.deserializeBinaryBulkWithMultipleStreams(column, input_stream_getter, rows, avg_value_size_hint, false, {}); + auto state = type.createDeserializeBinaryBulkState(); + type.deserializeBinaryBulkWithMultipleStreams(column, input_stream_getter, rows, avg_value_size_hint, false, {}, state); if (column.size() != rows) throw Exception("Cannot read all data in NativeBlockInputStream.", ErrorCodes::CANNOT_READ_ALL_DATA); diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 88ebe65202a..247d3fc193f 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -206,7 +206,8 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreams( size_t limit, double /*avg_value_size_hint*/, bool position_independent_encoding, - SubstreamPath path) const + SubstreamPath path, + const DeserializeBinaryBulkStatePtr & state) const { ColumnArray & column_array = typeid_cast(column); @@ -229,7 +230,7 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreams( if (last_offset < nested_column.size()) throw Exception("Nested column is longer than last offset", ErrorCodes::LOGICAL_ERROR); size_t nested_limit = last_offset - nested_column.size(); - nested->deserializeBinaryBulkWithMultipleStreams(nested_column, getter, nested_limit, 0, position_independent_encoding, path); + nested->deserializeBinaryBulkWithMultipleStreams(nested_column, getter, nested_limit, 0, position_independent_encoding, path, state); /// Check consistency between offsets and elements subcolumns. /// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER. diff --git a/dbms/src/DataTypes/DataTypeArray.h b/dbms/src/DataTypes/DataTypeArray.h index 668eb650a0b..83df57f69d6 100644 --- a/dbms/src/DataTypes/DataTypeArray.h +++ b/dbms/src/DataTypes/DataTypeArray.h @@ -71,13 +71,19 @@ public: bool position_independent_encoding, SubstreamPath path) const override; + DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const override + { + return nested->createDeserializeBinaryBulkState(); + } + void deserializeBinaryBulkWithMultipleStreams( IColumn & column, InputStreamGetter getter, size_t limit, double avg_value_size_hint, bool position_independent_encoding, - SubstreamPath path) const override; + SubstreamPath path, + const DeserializeBinaryBulkStatePtr & state) const override; MutableColumnPtr createColumn() const override; diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index 751f32eb462..0f532cab258 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -74,7 +74,8 @@ void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( size_t limit, double avg_value_size_hint, bool position_independent_encoding, - SubstreamPath path) const + SubstreamPath path, + const DeserializeBinaryBulkStatePtr & state) const { ColumnNullable & col = static_cast(column); @@ -83,7 +84,7 @@ void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0); path.back() = Substream::NullableElements; - nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, limit, avg_value_size_hint, position_independent_encoding, path); + nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, limit, avg_value_size_hint, position_independent_encoding, path, state); } diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index 5a3be2c4c05..d96692bed68 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -27,13 +27,19 @@ public: bool position_independent_encoding, SubstreamPath path) const override; + DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const override + { + return nested_data_type->createDeserializeBinaryBulkState(); + } + void deserializeBinaryBulkWithMultipleStreams( IColumn & column, InputStreamGetter getter, size_t limit, double avg_value_size_hint, bool position_independent_encoding, - SubstreamPath path) const override; + SubstreamPath path, + const DeserializeBinaryBulkStatePtr & state) const override; void serializeBinary(const Field & field, WriteBuffer & ostr) const override { nested_data_type->serializeBinary(field, ostr); } void deserializeBinary(Field & field, ReadBuffer & istr) const override { nested_data_type->deserializeBinary(field, istr); } diff --git a/dbms/src/DataTypes/DataTypeTuple.cpp b/dbms/src/DataTypes/DataTypeTuple.cpp index 3bcd815686f..f66ad76c85d 100644 --- a/dbms/src/DataTypes/DataTypeTuple.cpp +++ b/dbms/src/DataTypes/DataTypeTuple.cpp @@ -315,14 +315,18 @@ void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams( size_t limit, double avg_value_size_hint, bool position_independent_encoding, - SubstreamPath path) const + SubstreamPath path, + const DeserializeBinaryBulkStatePtr & state) const { + auto * tuple_state = typeid_cast(state.get()); + path.push_back(Substream::TupleElement); for (const auto i : ext::range(0, ext::size(elems))) { path.back().tuple_element_name = names[i]; elems[i]->deserializeBinaryBulkWithMultipleStreams( - extractElementColumn(column, i), getter, limit, avg_value_size_hint, position_independent_encoding, path); + extractElementColumn(column, i), getter, limit, avg_value_size_hint, + position_independent_encoding, path, tuple_state->states[i]); } } diff --git a/dbms/src/DataTypes/DataTypeTuple.h b/dbms/src/DataTypes/DataTypeTuple.h index 966950e1488..aeabd26f5a2 100644 --- a/dbms/src/DataTypes/DataTypeTuple.h +++ b/dbms/src/DataTypes/DataTypeTuple.h @@ -63,13 +63,30 @@ public: bool position_independent_encoding, SubstreamPath path) const override; + struct DeserializeBinaryBulkStateTuple : public IDataType::DeserializeBinaryBulkState + { + std::vector states; + DeserializeBinaryBulkStateTuple(const DataTypes & types) + { + states.reserve(types.size()); + for (auto & type : types) + states.emplace_back(type->createDeserializeBinaryBulkState()); + } + }; + + DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const override + { + return std::make_shared(elems); + } + void deserializeBinaryBulkWithMultipleStreams( IColumn & column, InputStreamGetter getter, size_t limit, double avg_value_size_hint, bool position_independent_encoding, - SubstreamPath path) const override; + SubstreamPath path, + const DeserializeBinaryBulkStatePtr & state) const override; MutableColumnPtr createColumn() const override; diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 199904a98c7..43b0e0afde4 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -69,17 +69,16 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( SubstreamPath path) const { const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); - MutableColumnPtr sub_index; size_t max_limit = column.size() - offset; limit = limit ? std::min(limit, max_limit) : max_limit; - path.push_back(Substream::DictionaryKeys); + path.push_back(Substream::DictionaryIndexes); if (auto stream = getter(path)) { const auto & indexes = column_with_dictionary.getIndexesPtr(); const auto & keys = column_with_dictionary.getUnique()->getNestedColumn(); - sub_index = (*indexes->cut(offset, limit)).mutate(); + MutableColumnPtr sub_index = (*indexes->cut(offset, limit)).mutate(); ColumnPtr unique_indexes = makeSubIndex(*sub_index); /// unique_indexes->index(sub_index) == indexes[offset:offset + limit] auto used_keys = keys->index(unique_indexes, 0); @@ -87,50 +86,88 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( UInt64 used_keys_size = used_keys->size(); writeIntBinary(used_keys_size, *stream); + + UInt64 indexes_size = sub_index->size(); + writeIntBinary(indexes_size, *stream); + + path.back() = Substream::DictionaryKeys; dictionary_type->serializeBinaryBulkWithMultipleStreams(*used_keys, getter, 0, 0, position_independent_encoding, path); - } - path.back() = Substream::DictionaryIndexes; - if (auto stream = getter(path)) - { - if (!sub_index) - throw Exception("Dictionary keys wasn't serialized", ErrorCodes::LOGICAL_ERROR); indexes_type->serializeBinaryBulk(*sub_index, *stream, 0, limit); } } +struct DeserializeBinaryBulkStateWithDictionary : public IDataType::DeserializeBinaryBulkState +{ + UInt64 num_rows_to_read_until_next_index = 0; + ColumnPtr index; + IDataType::DeserializeBinaryBulkStatePtr state; + + explicit DeserializeBinaryBulkStateWithDictionary(IDataType::DeserializeBinaryBulkStatePtr && state) + : state(std::move(state)) {} +}; + +IDataType::DeserializeBinaryBulkStatePtr DataTypeWithDictionary::createDeserializeBinaryBulkState() const +{ + return std::make_shared( + dictionary_type->createDeserializeBinaryBulkState()); +} + void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( IColumn & column, InputStreamGetter getter, size_t limit, double /*avg_value_size_hint*/, bool position_independent_encoding, - SubstreamPath path) const + SubstreamPath path, + const DeserializeBinaryBulkStatePtr & state) const { ColumnWithDictionary & column_with_dictionary = typeid_cast(column); - ColumnPtr indexes; - path.push_back(Substream::DictionaryKeys); - if (ReadBuffer * stream = getter(path)) + auto dict_state = typeid_cast(state.get()); + if (dict_state == nullptr) + throw Exception("Invalid DeserializeBinaryBulkState.", ErrorCodes::LOGICAL_ERROR); + + auto readIndexes = [&](ReadBuffer * stream, const ColumnPtr & index, size_t num_rows) { - UInt64 num_keys; - readIntBinary(num_keys, *stream); - auto dict_column = dictionary_type->createColumn(); - dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, getter, num_keys, 0, position_independent_encoding, path); - indexes = column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, num_keys); - } + auto index_col = indexes_type->createColumn(); + indexes_type->deserializeBinaryBulk(*index_col, *stream, num_rows, 0); + column_with_dictionary.getIndexes()->insertRangeFrom(*index->index(std::move(index_col), 0), 0, num_rows); + }; + + auto readDict = [&](UInt64 num_keys) + { + auto dict_column = dictionary_type->createColumn(); + dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, getter, num_keys, 0, + position_independent_encoding, path, dict_state->state); + return column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, num_keys); + }; + + path.push_back(Substream::DictionaryIndexes); - path.back() = Substream::DictionaryIndexes; if (auto stream = getter(path)) { - if (!indexes) - throw Exception("Dictionary keys wasn't deserialized", ErrorCodes::LOGICAL_ERROR); + path.back() = Substream::DictionaryKeys; - auto index_col = indexes_type->createColumn(); - indexes_type->deserializeBinaryBulk(*index_col, *stream, limit, 0); - auto index_size = index_col->size(); - column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(std::move(index_col), 0), 0, index_size); + while (limit) + { + if (dict_state->num_rows_to_read_until_next_index == 0) + { + if (stream->eof()) + break; + + UInt64 num_keys; + readIntBinary(num_keys, *stream); + readIntBinary(dict_state->num_rows_to_read_until_next_index, *stream); + dict_state->index = readDict(num_keys); + } + + size_t num_rows_to_read = std::min(limit, dict_state->num_rows_to_read_until_next_index); + readIndexes(stream, dict_state->index, num_rows_to_read); + limit -= num_rows_to_read; + dict_state->num_rows_to_read_until_next_index -= num_rows_to_read; + } } } diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index a76ca22cd33..8492a1c8745 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -29,16 +29,19 @@ public: OutputStreamGetter getter, size_t offset, size_t limit, - bool /*position_independent_encoding*/, + bool position_independent_encoding, SubstreamPath path) const override; + DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const override; + void deserializeBinaryBulkWithMultipleStreams( IColumn & column, InputStreamGetter getter, size_t limit, - double /*avg_value_size_hint*/, - bool /*position_independent_encoding*/, - SubstreamPath path) const override; + double avg_value_size_hint, + bool position_independent_encoding, + SubstreamPath path, + const DeserializeBinaryBulkStatePtr & state) const override; void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void deserializeBinary(Field & field, ReadBuffer & istr) const override; diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 5fa04ae2ef1..1751ec9a3b1 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -120,6 +120,14 @@ public: serializeBinaryBulk(column, *stream, offset, limit); } + struct DeserializeBinaryBulkState + { + virtual ~DeserializeBinaryBulkState() = default; + }; + using DeserializeBinaryBulkStatePtr = std::shared_ptr; + + virtual DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const { return nullptr; } + /** Read no more than limit values and append them into column. * avg_value_size_hint - if not zero, may be used to avoid reallocations while reading column of String type. */ @@ -129,7 +137,8 @@ public: size_t limit, double avg_value_size_hint, bool /*position_independent_encoding*/, - SubstreamPath path) const + SubstreamPath path, + const DeserializeBinaryBulkStatePtr & /*state*/) const { if (ReadBuffer * stream = getter(path)) deserializeBinaryBulk(column, *stream, limit, avg_value_size_hint); diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index ebc32c902cd..0129254679b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -382,18 +382,16 @@ void MergeTreeReader::readData( return stream.data_buffer; }; + if (!continue_reading) + deserialize_binary_bulk_state_map[name] = type.createDeserializeBinaryBulkState(); + + if (deserialize_binary_bulk_state_map.count(name) == 0) + throw Exception("DeserializeBinaryBulkState wasn't created for column " + name, ErrorCodes::LOGICAL_ERROR); + double & avg_value_size_hint = avg_value_size_hints[name]; - if (column.withDictionary()) - { - for (size_t read_rows = 0; read_rows < max_rows_to_read; read_rows += index_granularity) - { - size_t rows_to_read = std::min(index_granularity, max_rows_to_read - read_rows); - type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, rows_to_read, avg_value_size_hint, true, {}); - continue_reading = true; - } - } - else - type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, avg_value_size_hint, true, {}); + auto & deserialize_state = deserialize_binary_bulk_state_map[name]; + type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, + avg_value_size_hint, true, {}, deserialize_state); IDataType::updateAvgValueSizeHint(column, avg_value_size_hint); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 05ef06b7419..c473974709c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -22,6 +22,7 @@ class MergeTreeReader : private boost::noncopyable { public: using ValueSizeMap = std::map; + using DeserializeBinaryBulkStateMap = std::map; MergeTreeReader(const String & path, /// Path to the directory containing the part const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns, @@ -90,6 +91,8 @@ private: /// avg_value_size_hints are used to reduce the number of reallocations when creating columns of variable size. ValueSizeMap avg_value_size_hints; + /// Stores states for IDataType::deserializeBinaryBulk + DeserializeBinaryBulkStateMap deserialize_binary_bulk_state_map; String path; MergeTreeData::DataPartPtr data_part; diff --git a/dbms/src/Storages/MergeTree/checkDataPart.cpp b/dbms/src/Storages/MergeTree/checkDataPart.cpp index ff36619756b..9cf89bb1077 100644 --- a/dbms/src/Storages/MergeTree/checkDataPart.cpp +++ b/dbms/src/Storages/MergeTree/checkDataPart.cpp @@ -278,6 +278,7 @@ MergeTreeData::DataPart::Checksums checkDataPart( /// NOTE Shared array sizes of Nested columns are read more than once. That's Ok. MutableColumnPtr tmp_column = name_type.type->createColumn(); + auto state = name_type.type->createDeserializeBinaryBulkState(); name_type.type->deserializeBinaryBulkWithMultipleStreams( *tmp_column, [&](const IDataType::SubstreamPath & substream_path) @@ -289,7 +290,7 @@ MergeTreeData::DataPart::Checksums checkDataPart( return &stream_it->second.uncompressed_hashing_buf; }, index_granularity, - 0, true, {}); + 0, true, {}, state); size_t read_size = tmp_column->size(); column_size += read_size; diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 04d9f18ffd1..54c1323fcc7 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -243,7 +243,8 @@ void LogBlockInputStream::readData(const String & name, const IDataType & type, return &it->second.compressed; }; - type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, 0, true, {}); /// TODO Use avg_value_size_hint. + auto state = type.createDeserializeBinaryBulkState(); + type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, 0, true, {}, state); /// TODO Use avg_value_size_hint. } diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 1a0cb0c2179..ab4758a2981 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -216,7 +216,8 @@ void TinyLogBlockInputStream::readData(const String & name, const IDataType & ty return &streams[stream_name]->compressed; }; - type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, limit, 0, true, {}); /// TODO Use avg_value_size_hint. + auto state = type.createDeserializeBinaryBulkState(); + type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, limit, 0, true, {}, state); /// TODO Use avg_value_size_hint. } From 96df20f6b50b6cd9dafed7f0b209a15a46ff0caf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Jun 2018 14:20:27 +0300 Subject: [PATCH 092/315] Fixed ColumnUnique::compareAt implementation; deprecated several metods for IColumnUnique --- dbms/src/Columns/ColumnUnique.h | 61 ++++++++++++------------- dbms/src/Columns/ColumnWithDictionary.h | 2 +- dbms/src/Columns/IColumnUnique.h | 31 ++++++++++++- 3 files changed, 61 insertions(+), 33 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index bc579829bc3..4d0299588bb 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -56,7 +56,9 @@ class ColumnUnique final : public COWPtrHelperget(n, res); } - StringRef getDataAt(size_t n) const override { return column_holder->getDataAt(n); } + Field operator[](size_t n) const override { return (*getNestedColumn())[n]; } + void get(size_t n, Field & res) const override { getNestedColumn()->get(n, res); } + StringRef getDataAt(size_t n) const override { return getNestedColumn()->getDataAt(n); } StringRef getDataAtWithTerminatingZero(size_t n) const override { - return column_holder->getDataAtWithTerminatingZero(n); + return getNestedColumn()->getDataAtWithTerminatingZero(n); } - UInt64 get64(size_t n) const override { return column_holder->get64(n); } - UInt64 getUInt(size_t n) const override { return column_holder->getUInt(n); } - Int64 getInt(size_t n) const override { return column_holder->getInt(n); } - bool isNullAt(size_t n) const override { return column_holder->isNullAt(n); } - ColumnPtr cut(size_t start, size_t length) const override { return column_holder->cut(start, length); } + UInt64 get64(size_t n) const override { return getNestedColumn()->get64(n); } + UInt64 getUInt(size_t n) const override { return getNestedColumn()->getUInt(n); } + Int64 getInt(size_t n) const override { return getNestedColumn()->getInt(n); } + bool isNullAt(size_t n) const override { return getNestedColumn()->isNullAt(n); } StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override { return column_holder->serializeValueIntoArena(n, arena, begin); } void updateHashWithValue(size_t n, SipHash & hash) const override { - return column_holder->updateHashWithValue(n, hash); - } - ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override - { - return column_holder->filter(filt, result_size_hint); - } - ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override - { - return column_holder->permute(perm, limit); + return getNestedColumn()->updateHashWithValue(n, hash); } + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override { - return column_holder->compareAt(n, m, rhs, nan_direction_hint); - } - void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override - { - column_holder->getPermutation(reverse, limit, nan_direction_hint, res); - } - ColumnPtr replicate(const IColumn::Offsets & offsets) const override { return column_holder->replicate(offsets); } - std::vector scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const override - { - return column_holder->scatter(num_columns, selector); + auto & column_unique = static_cast(rhs); + return getNestedColumn()->compareAt(n, m, *column_unique.getNestedColumn(), nan_direction_hint); } + void getExtremes(Field & min, Field & max) const override { column_holder->getExtremes(min, max); } bool valuesHaveFixedSize() const override { return column_holder->valuesHaveFixedSize(); } bool isFixedAndContiguous() const override { return column_holder->isFixedAndContiguous(); } @@ -121,9 +108,21 @@ public: size_t byteSize() const override { return column_holder->byteSize(); } size_t allocatedBytes() const override { - return column_holder->allocatedBytes() + (index ? index->getBufferSizeInBytes() : 0); + return column_holder->allocatedBytes() + + (index ? index->getBufferSizeInBytes() : 0) + + (nullable_column ? nullable_column->allocatedBytes() : 0); + } + void forEachSubcolumn(IColumn::ColumnCallback callback) override + { + callback(is_nullable ? nullable_column : column_holder); + /// If column was mutated, we need to restore ptrs. + if (is_nullable) + { + auto & column_nullable = static_cast(nullable_column->assumeMutableRef()); + column_holder = column_nullable.getNestedColumnPtr(); + nullable_column_map = &column_nullable.getNullMapData(); + } } - void forEachSubcolumn(IColumn::ColumnCallback callback) override { callback(column_holder); } private: diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index b202f074719..65e8572774a 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -165,7 +165,7 @@ public: { size_t unique_limit = std::min(limit, getUnique()->size()); Permutation unique_perm; - getUnique()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); + getUnique()->getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); /// TODO: optimize with sse. diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index 36ba42d1889..9aba22d068e 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -37,7 +37,6 @@ public: // // virtual bool has(const char * pos, size_t length) const { return getInsertionPoint(pos, length) != size(); } - const char * getFamilyName() const override { return "ColumnUnique"; } void insert(const Field &) override @@ -79,6 +78,36 @@ public: { throw Exception("Method index is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } + + ColumnPtr cut(size_t, size_t) const override + { + throw Exception("Method cut is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + ColumnPtr filter(const IColumn::Filter &, ssize_t) const override + { + throw Exception("Method filter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + ColumnPtr permute(const IColumn::Permutation &, size_t) const override + { + throw Exception("Method permute is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + ColumnPtr replicate(const IColumn::Offsets &) const override + { + throw Exception("Method replicate is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + void getPermutation(bool, size_t, int, IColumn::Permutation &) const override + { + throw Exception("Method getPermutation is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } + + std::vector scatter(IColumn::ColumnIndex, const IColumn::Selector &) const override + { + throw Exception("Method scatter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } }; } From 12bf5ef5657f37d5bc9e1e692eb28bec7f34bad9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Jun 2018 16:43:16 +0300 Subject: [PATCH 093/315] Added IColumn::convertToFullColumnIfWithDictionary; supported FilterDescription for ColumnWithDictionary --- dbms/src/Columns/ColumnUnique.h | 2 +- dbms/src/Columns/ColumnWithDictionary.h | 2 ++ dbms/src/Columns/FilterDescription.cpp | 15 ++++++++++----- dbms/src/Columns/IColumn.h | 4 ++++ dbms/src/DataTypes/DataTypeWithDictionary.cpp | 14 +++++++++++++- 5 files changed, 30 insertions(+), 7 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 4d0299588bb..255cbe85078 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -83,7 +83,7 @@ public: UInt64 get64(size_t n) const override { return getNestedColumn()->get64(n); } UInt64 getUInt(size_t n) const override { return getNestedColumn()->getUInt(n); } Int64 getInt(size_t n) const override { return getNestedColumn()->getInt(n); } - bool isNullAt(size_t n) const override { return getNestedColumn()->isNullAt(n); } + bool isNullAt(size_t n) const override { return is_nullable && n == getNullValueIndex(); } StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override { return column_holder->serializeValueIntoArena(n, arena, begin); diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index 65e8572774a..a4e584be1fd 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -42,6 +42,8 @@ public: return getUnique()->getNestedColumn()->index(indexes, 0); } + ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); } + MutableColumnPtr cloneResized(size_t size) const override { auto unique_ptr = column_unique; diff --git a/dbms/src/Columns/FilterDescription.cpp b/dbms/src/Columns/FilterDescription.cpp index b4779648b82..92c2765feda 100644 --- a/dbms/src/Columns/FilterDescription.cpp +++ b/dbms/src/Columns/FilterDescription.cpp @@ -27,14 +27,14 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) if (column.isColumnConst()) { const ColumnConst & column_const = static_cast(column); - const IColumn & column_nested = column_const.getDataColumn(); + ColumnPtr column_nested = column_const.getDataColumnPtr()->convertToFullColumnIfWithDictionary(); - if (!typeid_cast(&column_nested)) + if (!typeid_cast(column_nested.get())) { - const ColumnNullable * column_nested_nullable = typeid_cast(&column_nested); + const ColumnNullable * column_nested_nullable = typeid_cast(column_nested.get()); if (!column_nested_nullable || !typeid_cast(&column_nested_nullable->getNestedColumn())) { - throw Exception("Illegal type " + column_nested.getName() + " of column for constant filter. Must be UInt8 or Nullable(UInt8).", + throw Exception("Illegal type " + column_nested->getName() + " of column for constant filter. Must be UInt8 or Nullable(UInt8).", ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); } } @@ -48,8 +48,13 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) } -FilterDescription::FilterDescription(const IColumn & column) +FilterDescription::FilterDescription(const IColumn & column_) { + if (column_.withDictionary()) + data_holder = column_.convertToFullColumnIfWithDictionary(); + + const auto & column = data_holder ? *data_holder : column_; + if (const ColumnUInt8 * concrete_column = typeid_cast(&column)) { data = &concrete_column->getData(); diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 059dc1e9e0c..28fcbadee6f 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -47,6 +47,10 @@ public: */ virtual Ptr convertToFullColumnIfConst() const { return {}; } + /// If column isn't ColumnWithDictionary, return itself. + /// If column is ColumnWithDictionary, transforms is to full column. + virtual Ptr convertToFullColumnIfWithDictionary() const { return getPtr(); } + /// Creates empty column with the same type. virtual MutablePtr cloneEmpty() const { return cloneResized(0); } diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 43b0e0afde4..c0bf334eb18 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -136,10 +136,22 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( column_with_dictionary.getIndexes()->insertRangeFrom(*index->index(std::move(index_col), 0), 0, num_rows); }; + using CachedStreams = std::unordered_map; + CachedStreams cached_streams; + + IDataType::InputStreamGetter cached_stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer * + { + std::string stream_name = IDataType::getFileNameForStream("", path); + auto iter = cached_streams.find(stream_name); + if (iter == cached_streams.end()) + iter = cached_streams.insert({stream_name, getter(path)}).first; + return iter.second; + }; + auto readDict = [&](UInt64 num_keys) { auto dict_column = dictionary_type->createColumn(); - dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, getter, num_keys, 0, + dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, cached_stream_getter, num_keys, 0, position_independent_encoding, path, dict_state->state); return column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, num_keys); }; From df71a25b4f4a120d30d8b4b90db5c39e8daea9dc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Jun 2018 19:10:06 +0300 Subject: [PATCH 094/315] Removed function makeDictionary, added functions makeDictionaryUInt{8,16,32,64} --- dbms/src/Functions/FunctionsMiscellaneous.cpp | 35 ++++++++++++++++--- 1 file changed, 31 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index 97c11e8dcb8..592d70893cd 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -1832,11 +1832,35 @@ public: }; +template +struct FunctionMakeDictionaryName; +template <> +struct FunctionMakeDictionaryName +{ + static constexpr auto name = "makeDictionaryUInt8"; +}; +template <> +struct FunctionMakeDictionaryName +{ + static constexpr auto name = "makeDictionaryUInt16"; +}; +template <> +struct FunctionMakeDictionaryName +{ + static constexpr auto name = "makeDictionaryUInt32"; +}; +template <> +struct FunctionMakeDictionaryName +{ + static constexpr auto name = "makeDictionaryUInt64"; +}; + +template class FunctionMakeDictionary: public IFunction { public: - static constexpr auto name = "makeDictionary"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static constexpr auto name = FunctionMakeDictionaryName::name; + static FunctionPtr create(const Context &) { return std::make_shared>(); } String getName() const override { return name; } @@ -1847,7 +1871,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - return std::make_shared(arguments[0], std::make_shared()); + return std::make_shared(arguments[0], std::make_shared>()); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override @@ -1981,7 +2005,10 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); factory.registerFunction(); factory.registerFunction(); } From 58cbcbd5c33fe87f37148847f6eb44bd9d4e23f6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Jun 2018 13:37:06 +0300 Subject: [PATCH 095/315] Supported DataTypeWithDictionary in CAST function. --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 2 +- dbms/src/Functions/FunctionsConversion.h | 96 ++++++++++++++++--- 2 files changed, 86 insertions(+), 12 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index c0bf334eb18..9ec91eb4a68 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -145,7 +145,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( auto iter = cached_streams.find(stream_name); if (iter == cached_streams.end()) iter = cached_streams.insert({stream_name, getter(path)}).first; - return iter.second; + return iter->second; }; auto readDict = [&](UInt64 num_keys) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index d9fcd788b27..2ea6879f31c 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -35,6 +35,8 @@ #include #include #include +#include +#include namespace DB @@ -1239,7 +1241,8 @@ public: PreparedFunctionPtr prepare(const Block & /*sample_block*/) const override { - return std::make_shared(prepare(getArgumentTypes()[0], getReturnType()), name); + return std::make_shared( + prepareUnpackDictionaries(getArgumentTypes()[0], getReturnType()), name); } String getName() const override { return name; } @@ -1322,7 +1325,7 @@ private: throw Exception{"CAST AS Array can only be performed between same-dimensional array types or from String", ErrorCodes::TYPE_MISMATCH}; /// Prepare nested type conversion - const auto nested_function = prepare(from_nested_type, to_nested_type); + const auto nested_function = prepareUnpackDictionaries(from_nested_type, to_nested_type); return [nested_function, from_nested_type, to_nested_type]( Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/) @@ -1376,7 +1379,7 @@ private: /// Create conversion wrapper for each element in tuple for (const auto & idx_type : ext::enumerate(from_type->getElements())) - element_wrappers.push_back(prepare(idx_type.second, to_element_types[idx_type.first])); + element_wrappers.push_back(prepareUnpackDictionaries(idx_type.second, to_element_types[idx_type.first])); return [element_wrappers, from_element_types, to_element_types] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) @@ -1527,17 +1530,11 @@ private: bool result_is_nullable = false; }; - WrapperType prepare(const DataTypePtr & from_type, const DataTypePtr & to_type) const + WrapperType prepareUnpackDictionaries(const DataTypePtr & from_type, const DataTypePtr & to_type) const { - /// Determine whether pre-processing and/or post-processing must take place during conversion. - - NullableConversion nullable_conversion; - nullable_conversion.source_is_nullable = from_type->isNullable(); - nullable_conversion.result_is_nullable = to_type->isNullable(); - if (from_type->onlyNull()) { - if (!nullable_conversion.result_is_nullable) + if (!to_type->isNullable()) throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE}; return [](Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count) @@ -1547,6 +1544,83 @@ private: }; } + const auto * from_with_dict = typeid_cast(from_type.get()); + const auto * to_with_dict = typeid_cast(to_type.get()); + const auto & from_nested = from_with_dict ? from_with_dict->getDictionaryType() : from_type; + const auto & to_nested = to_with_dict ? to_with_dict->getDictionaryType() : to_type; + + auto wrapper = prepareRemoveNullable(from_nested, to_nested); + if (!from_with_dict && !to_with_dict) + return wrapper; + + return [wrapper, from_with_dict, to_with_dict] + (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) + { + auto & arg = block.getByPosition(arguments[0]); + auto & res = block.getByPosition(result); + + ColumnPtr res_indexes; + + { + /// Replace argument and result columns (and types) to dictionary key columns (and types). + /// Call nested wrapper in order to cast dictionary keys. Then restore block. + auto prev_arg_col = arg.column; + auto prev_arg_type = arg.type; + auto prev_res_type = res.type; + + auto tmp_rows_count = input_rows_count; + + if (from_with_dict) + { + auto * col_with_dict = typeid_cast(prev_arg_col.get()); + arg.column = col_with_dict->getUnique()->getNestedColumn(); + arg.type = from_with_dict->getDictionaryType(); + + tmp_rows_count = arg.column->size(); + res_indexes = col_with_dict->getIndexesPtr(); + } + + if (to_with_dict) + res.type = to_with_dict->getDictionaryType(); + + /// Perform the requested conversion. + wrapper(block, arguments, result, tmp_rows_count); + + arg.column = prev_arg_col; + arg.type = prev_arg_type; + res.type = prev_res_type; + } + + if (to_with_dict) + { + auto res_column = to_with_dict->createColumn(); + auto * col_with_dict = typeid_cast(res_column.get()); + + if (from_with_dict) + { + auto res_keys = std::move(res.column); + + auto idx = col_with_dict->getUnique()->uniqueInsertRangeFrom(*res_keys, 0, res_keys->size()); + col_with_dict->getIndexes()->insertRangeFrom(*idx->index(res_indexes, 0), 0, res_indexes->size()); + } + else + col_with_dict->insertRangeFromFullColumn(*res.column, 0, res.column->size()); + + res.column = std::move(res_column); + } + else + res.column = res.column->index(res_indexes, 0); + }; + } + + WrapperType prepareRemoveNullable(const DataTypePtr & from_type, const DataTypePtr & to_type) const + { + /// Determine whether pre-processing and/or post-processing must take place during conversion. + + NullableConversion nullable_conversion; + nullable_conversion.source_is_nullable = from_type->isNullable(); + nullable_conversion.result_is_nullable = to_type->isNullable(); + DataTypePtr from_inner_type = removeNullable(from_type); DataTypePtr to_inner_type = removeNullable(to_type); From f56d16769b86e52c1d200fb9dcfc4376c75f4565 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Jun 2018 21:14:37 +0300 Subject: [PATCH 096/315] Added pre and post serialization for IDataType BinaryBulkWithMultipleStreams. Supported dictionary serialization format with single global dictionary (limited wit max_dictionary_size) and additional keys which are stored per granule. Changed IDataType::enumerateStream interface. Added (de)serialization params for binary bulk with multiple stream (de)serialization. Changed IColumn::index interface. --- dbms/src/Columns/ColumnAggregateFunction.cpp | 2 +- dbms/src/Columns/ColumnAggregateFunction.h | 2 +- dbms/src/Columns/ColumnArray.cpp | 4 +- dbms/src/Columns/ColumnArray.h | 2 +- dbms/src/Columns/ColumnConst.cpp | 8 +- dbms/src/Columns/ColumnConst.h | 2 +- dbms/src/Columns/ColumnFixedString.cpp | 2 +- dbms/src/Columns/ColumnFixedString.h | 2 +- dbms/src/Columns/ColumnFunction.cpp | 2 +- dbms/src/Columns/ColumnFunction.h | 2 +- dbms/src/Columns/ColumnNullable.cpp | 2 +- dbms/src/Columns/ColumnNullable.h | 2 +- dbms/src/Columns/ColumnString.cpp | 2 +- dbms/src/Columns/ColumnString.h | 2 +- dbms/src/Columns/ColumnTuple.cpp | 2 +- dbms/src/Columns/ColumnTuple.h | 2 +- dbms/src/Columns/ColumnUnique.h | 106 ++- dbms/src/Columns/ColumnVector.cpp | 2 +- dbms/src/Columns/ColumnVector.h | 2 +- dbms/src/Columns/ColumnWithDictionary.cpp | 2 +- dbms/src/Columns/ColumnWithDictionary.h | 9 +- dbms/src/Columns/ColumnsCommon.cpp | 58 +- dbms/src/Columns/ColumnsCommon.h | 15 +- dbms/src/Columns/IColumn.h | 2 +- dbms/src/Columns/IColumnDummy.h | 4 +- dbms/src/Columns/IColumnUnique.h | 35 +- .../DataStreams/NativeBlockInputStream.cpp | 11 +- .../DataStreams/NativeBlockOutputStream.cpp | 11 +- dbms/src/DataTypes/DataTypeArray.cpp | 67 +- dbms/src/DataTypes/DataTypeArray.h | 41 +- dbms/src/DataTypes/DataTypeNullable.cpp | 63 +- dbms/src/DataTypes/DataTypeNullable.h | 41 +- dbms/src/DataTypes/DataTypeTuple.cpp | 127 +++- dbms/src/DataTypes/DataTypeTuple.h | 52 +- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 659 ++++++++++++++---- dbms/src/DataTypes/DataTypeWithDictionary.h | 36 +- dbms/src/DataTypes/IDataType.h | 80 ++- dbms/src/Functions/FunctionsConversion.h | 4 +- dbms/src/Functions/IFunction.cpp | 3 +- .../Storages/MergeTree/MergeTreeDataPart.cpp | 8 +- .../Storages/MergeTree/MergeTreeReader.cpp | 45 +- .../MergeTree/MergedBlockOutputStream.cpp | 124 +++- .../MergeTree/MergedBlockOutputStream.h | 10 +- dbms/src/Storages/MergeTree/checkDataPart.cpp | 29 +- dbms/src/Storages/StorageLog.cpp | 135 ++-- dbms/src/Storages/StorageTinyLog.cpp | 53 +- 46 files changed, 1364 insertions(+), 510 deletions(-) diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index b04a8fbd68f..80b4b95fd4a 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -162,7 +162,7 @@ ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limi return std::move(res); } -ColumnPtr ColumnAggregateFunction::index(const ColumnPtr & indexes, size_t limit) const +ColumnPtr ColumnAggregateFunction::index(const IColumn & indexes, size_t limit) const { return selectIndexImpl(*this, indexes, limit); } diff --git a/dbms/src/Columns/ColumnAggregateFunction.h b/dbms/src/Columns/ColumnAggregateFunction.h index c0b64b69cda..d15658c893f 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.h +++ b/dbms/src/Columns/ColumnAggregateFunction.h @@ -156,7 +156,7 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 085c21673f4..e3808a73d55 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -626,7 +626,7 @@ ColumnPtr ColumnArray::permute(const Permutation & perm, size_t limit) const return std::move(res); } -ColumnPtr ColumnArray::index(const ColumnPtr & indexes, size_t limit) const +ColumnPtr ColumnArray::index(const IColumn & indexes, size_t limit) const { return selectIndexImpl(*this, indexes, limit); } @@ -657,7 +657,7 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray & indexes, size_t limit } if (current_offset != 0) - res->data = data->index(std::move(nested_indexes_column), current_offset); + res->data = data->index(*nested_indexes_column, current_offset); return std::move(res); } diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index 1885a396a15..642d3da2327 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -71,7 +71,7 @@ public: void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index a0f7bc43567..607b6651499 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -63,13 +63,13 @@ ColumnPtr ColumnConst::permute(const Permutation & perm, size_t limit) const return ColumnConst::create(data, limit); } -ColumnPtr ColumnConst::index(const ColumnPtr & indexes, size_t limit) const +ColumnPtr ColumnConst::index(const IColumn & indexes, size_t limit) const { if (limit == 0) - limit = indexes->size(); + limit = indexes.size(); - if (indexes->size() < limit) - throw Exception("Size of indexes (" + toString(indexes->size()) + ") is less than required (" + toString(limit) + ")", + if (indexes.size() < limit) + throw Exception("Size of indexes (" + toString(indexes.size()) + ") is less than required (" + toString(limit) + ")", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); return ColumnConst::create(data, limit); diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index 241daa81467..d85c00eb604 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -153,7 +153,7 @@ public: ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr replicate(const Offsets & offsets) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; size_t byteSize() const override diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index 93f276a50ba..d110ef10a84 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -260,7 +260,7 @@ ColumnPtr ColumnFixedString::permute(const Permutation & perm, size_t limit) con } -ColumnPtr ColumnFixedString::index(const ColumnPtr & indexes, size_t limit) const +ColumnPtr ColumnFixedString::index(const IColumn & indexes, size_t limit) const { return selectIndexImpl(*this, indexes, limit); } diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index c98e8ce7c69..355af7ac3ba 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -108,7 +108,7 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; diff --git a/dbms/src/Columns/ColumnFunction.cpp b/dbms/src/Columns/ColumnFunction.cpp index 763b6b008b4..f58160a1b84 100644 --- a/dbms/src/Columns/ColumnFunction.cpp +++ b/dbms/src/Columns/ColumnFunction.cpp @@ -86,7 +86,7 @@ ColumnPtr ColumnFunction::permute(const Permutation & perm, size_t limit) const return ColumnFunction::create(limit, function, capture); } -ColumnPtr ColumnFunction::index(const ColumnPtr & indexes, size_t limit) const +ColumnPtr ColumnFunction::index(const IColumn & indexes, size_t limit) const { ColumnsWithTypeAndName capture = captured_columns; for (auto & column : capture) diff --git a/dbms/src/Columns/ColumnFunction.h b/dbms/src/Columns/ColumnFunction.h index 24ef11df07f..458167ffefe 100644 --- a/dbms/src/Columns/ColumnFunction.h +++ b/dbms/src/Columns/ColumnFunction.h @@ -33,7 +33,7 @@ public: ColumnPtr replicate(const Offsets & offsets) const override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; void insertDefault() override; void popBack(size_t n) override; std::vector scatter(IColumn::ColumnIndex num_columns, diff --git a/dbms/src/Columns/ColumnNullable.cpp b/dbms/src/Columns/ColumnNullable.cpp index 5f1c5f5bab7..5235bd23d04 100644 --- a/dbms/src/Columns/ColumnNullable.cpp +++ b/dbms/src/Columns/ColumnNullable.cpp @@ -166,7 +166,7 @@ ColumnPtr ColumnNullable::permute(const Permutation & perm, size_t limit) const return ColumnNullable::create(permuted_data, permuted_null_map); } -ColumnPtr ColumnNullable::index(const ColumnPtr & indexes, size_t limit) const +ColumnPtr ColumnNullable::index(const IColumn & indexes, size_t limit) const { ColumnPtr indexed_data = getNestedColumn().index(indexes, limit); ColumnPtr indexed_null_map = getNullMapColumn().index(indexes, limit); diff --git a/dbms/src/Columns/ColumnNullable.h b/dbms/src/Columns/ColumnNullable.h index ea96046520a..483a3a42e6f 100644 --- a/dbms/src/Columns/ColumnNullable.h +++ b/dbms/src/Columns/ColumnNullable.h @@ -65,7 +65,7 @@ public: void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void reserve(size_t n) override; diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index 93bc7d944a0..6d53dd10ff6 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -162,7 +162,7 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const } -ColumnPtr ColumnString::index(const ColumnPtr & indexes, size_t limit) const +ColumnPtr ColumnString::index(const IColumn & indexes, size_t limit) const { return selectIndexImpl(*this, indexes, limit); } diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index d29d8abc0e3..9d00d4b94c2 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -220,7 +220,7 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; diff --git a/dbms/src/Columns/ColumnTuple.cpp b/dbms/src/Columns/ColumnTuple.cpp index b2ae4264935..c4c93cf8def 100644 --- a/dbms/src/Columns/ColumnTuple.cpp +++ b/dbms/src/Columns/ColumnTuple.cpp @@ -179,7 +179,7 @@ ColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) const return ColumnTuple::create(new_columns); } -ColumnPtr ColumnTuple::index(const ColumnPtr & indexes, size_t limit) const +ColumnPtr ColumnTuple::index(const IColumn & indexes, size_t limit) const { const size_t tuple_size = columns.size(); Columns new_columns(tuple_size); diff --git a/dbms/src/Columns/ColumnTuple.h b/dbms/src/Columns/ColumnTuple.h index 648914219f9..b9fefd26db4 100644 --- a/dbms/src/Columns/ColumnTuple.h +++ b/dbms/src/Columns/ColumnTuple.h @@ -60,7 +60,7 @@ public: void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; void gather(ColumnGathererStream & gatherer_stream) override; diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 255cbe85078..06c7f10a184 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -55,7 +55,7 @@ class ColumnUnique final : public COWPtrHelper(column_holder.get()); } IndexType insertIntoMap(const StringRefWrapper & ref, IndexType value); + void uniqueInsertRangeImpl( + const IColumn & src, + size_t start, + size_t length, + typename ColumnVector::Container & positions, + ColumnType * overflowed_keys, + size_t max_dictionary_size); }; template -ColumnUnique::ColumnUnique(const DataTypePtr & type) : is_nullable(type->isNullable()) +ColumnUnique::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable()) { if (is_nullable) { - nullable_column = type->createColumn()->cloneResized(numSpecialValues()); + nullable_column = type.createColumn()->cloneResized(numSpecialValues()); auto & column_nullable = static_cast(nullable_column->assumeMutableRef()); column_holder = column_nullable.getNestedColumnPtr(); nullable_column_map = &column_nullable.getNullMapData(); (*nullable_column_map)[getDefaultValueIndex()] = 0; } else - column_holder = type->createColumn()->cloneResized(numSpecialValues()); + column_holder = type.createColumn()->cloneResized(numSpecialValues()); } template @@ -330,7 +340,13 @@ size_t ColumnUnique::uniqueDeserializeAndInsertFromArena( } template -ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnUnique::uniqueInsertRangeImpl( + const IColumn & src, + size_t start, + size_t length, + typename ColumnVector::Container & positions, + ColumnType * overflowed_keys, + size_t max_dictionary_size) { if (!index) buildIndex(); @@ -347,9 +363,11 @@ ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColu else src_column = static_cast(&src); + std::unique_ptr secondary_index; + if (overflowed_keys) + secondary_index = std::make_unique(); + auto column = getRawColumnPtr(); - auto positions_column = ColumnVector::create(length); - auto & positions = positions_column->getData(); size_t next_position = column->size(); for (auto i : ext::range(0, length)) @@ -365,18 +383,80 @@ ColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColu auto it = index->find(StringRefWrapper(src_column, row)); if (it == index->end()) { - positions[i] = next_position; - auto ref = src_column->getDataAt(row); - column->insertData(ref.data, ref.size); - (*index)[StringRefWrapper(column, next_position)] = next_position; - ++next_position; + + if (overflowed_keys && next_position >= max_dictionary_size + numSpecialValues()) + { + auto jt = secondary_index->find(StringRefWrapper(src_column, row)); + if (jt == secondary_index->end()) + { + positions[i] = next_position; + auto ref = src_column->getDataAt(row); + overflowed_keys->insertData(ref.data, ref.size); + (*secondary_index)[StringRefWrapper(src_column, row)] = next_position; + ++next_position; + } + else + positions[i] = jt->second; + } + else + { + positions[i] = next_position; + auto ref = src_column->getDataAt(row); + column->insertData(ref.data, ref.size); + (*index)[StringRefWrapper(column, next_position)] = next_position; + ++next_position; + } } else positions[i] = it->second; } } +} + +template +MutableColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) +{ + auto positions_column = ColumnVector::create(length); + auto & positions = positions_column->getData(); + + uniqueInsertRangeImpl(src, start, length, positions, nullptr, 0); return positions_column; } +template +IColumnUnique::IndexesWithOverflow ColumnUnique::uniqueInsertRangeWithOverflow( + const IColumn & src, + size_t start, + size_t length, + size_t max_dictionary_size) +{ + + auto positions_column = ColumnVector::create(length); + auto overflowed_keys = column_holder->cloneEmpty(); + auto & positions = positions_column->getData(); + + auto overflowed_keys_ptr = typeid_cast(overflowed_keys.get()); + if (!overflowed_keys_ptr) + throw Exception("Invalid keys type for ColumnUnique.", ErrorCodes::LOGICAL_ERROR); + + uniqueInsertRangeImpl(src, start, length, positions, overflowed_keys_ptr, max_dictionary_size); + + IColumnUnique::IndexesWithOverflow indexes_with_overflow; + indexes_with_overflow.indexes = std::move(positions_column); + indexes_with_overflow.overflowed_keys = std::move(overflowed_keys); + return indexes_with_overflow; } + +template +IColumnUnique::SerializableState ColumnUnique::getSerializableState() const +{ + IColumnUnique::SerializableState state; + state.column = column_holder; + state.offset = numSpecialValues(); + state.limit = column_holder->size() - state.offset; + + return state; +} + +}; diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 8aa9f463c52..4fc22d28e70 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -233,7 +233,7 @@ ColumnPtr ColumnVector::permute(const IColumn::Permutation & perm, size_t lim } template -ColumnPtr ColumnVector::index(const ColumnPtr & indexes, size_t limit) const +ColumnPtr ColumnVector::index(const IColumn & indexes, size_t limit) const { return selectIndexImpl(*this, indexes, limit); } diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 5a3bd336aa8..4800fe81115 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -252,7 +252,7 @@ public: ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override; - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; diff --git a/dbms/src/Columns/ColumnWithDictionary.cpp b/dbms/src/Columns/ColumnWithDictionary.cpp index 6adb07721bf..42bfd16dce2 100644 --- a/dbms/src/Columns/ColumnWithDictionary.cpp +++ b/dbms/src/Columns/ColumnWithDictionary.cpp @@ -5,7 +5,7 @@ namespace DB { ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_) - : column_unique(std::move(column_unique_)), indexes(std::move(indexes_)) + : column_unique(std::move(column_unique_)), indexes(std::move(indexes_)) { if (!dynamic_cast(column_unique.get())) throw Exception("ColumnUnique expected as argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index a4e584be1fd..7d665cc9a1c 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -39,7 +39,7 @@ public: ColumnPtr convertToFullColumn() const { - return getUnique()->getNestedColumn()->index(indexes, 0); + return getUnique()->getNestedColumn()->index(*indexes, 0); } ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); } @@ -102,7 +102,7 @@ public: auto & src_with_dict = static_cast(src); auto & src_nested = src_with_dict.getUnique()->getNestedColumn(); auto inserted_idx = getUnique()->uniqueInsertRangeFrom(*src_nested, 0, src_nested->size()); - auto idx = inserted_idx->index(src_with_dict.getIndexes()->cut(start, length), 0); + auto idx = inserted_idx->index(*src_with_dict.getIndexes()->cut(start, length), 0); getIndexes()->insertRangeFrom(*idx, 0, length); } @@ -150,7 +150,7 @@ public: return ColumnWithDictionary::create(column_unique, indexes->permute(perm, limit)); } - ColumnPtr index(const ColumnPtr & indexes_, size_t limit) const override + ColumnPtr index(const IColumn & indexes_, size_t limit) const override { return ColumnWithDictionary::create(column_unique, indexes->index(indexes_, limit)); } @@ -233,13 +233,14 @@ public: IColumnUnique * getUnique() { return static_cast(column_unique->assumeMutable().get()); } const IColumnUnique * getUnique() const { return static_cast(column_unique->assumeMutable().get()); } - const ColumnPtr & getUniquePtr() const { return column_unique; } + ColumnPtr getUniquePtr() const { return column_unique; } IColumn * getIndexes() { return indexes->assumeMutable().get(); } const IColumn * getIndexes() const { return indexes.get(); } const ColumnPtr & getIndexesPtr() const { return indexes; } void setIndexes(MutableColumnPtr && indexes_) { indexes = std::move(indexes_); } + void setUnique(const ColumnPtr & unique) { column_unique = unique; } bool withDictionary() const override { return true; } diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index 3eade6881ab..ed62028fa2e 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -315,65 +315,19 @@ INSTANTIATE(Float64) namespace detail { template - const PaddedPODArray * getIndexesData(const ColumnPtr & indexes) + const PaddedPODArray * getIndexesData(const IColumn & indexes) { - auto * column = typeid_cast *>(indexes.get()); + auto * column = typeid_cast *>(&indexes); if (column) return &column->getData(); return nullptr; } - template - PaddedPODArray * getIndexesData(IColumn & indexes) - { - auto * column = typeid_cast *>(&indexes); - if (column) - return &column->getData(); - - return nullptr; - } - - template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - template const PaddedPODArray * getIndexesData(const DB::ColumnPtr & indexes); - - template - MutableColumnPtr getUniqueIndexImpl(PaddedPODArray & index) - { - HashMap hash_map; - for (auto val : index) - hash_map.insert({val, hash_map.size()}); - - auto res_col = ColumnVector::create(); - auto & data = res_col->getData(); - - data.resize(hash_map.size()); - for (auto val : hash_map) - data[val.second] = val.first; - - for (auto & ind : index) - ind = hash_map[ind]; - - return std::move(res_col); - } -} - -/// Returns unique values of column. Write new index to column. -MutableColumnPtr makeSubIndex(IColumn & column) -{ - if (auto * data_uint8 = detail::getIndexesData(column)) - return detail::getUniqueIndexImpl(*data_uint8); - else if (auto * data_uint16 = detail::getIndexesData(column)) - return detail::getUniqueIndexImpl(*data_uint16); - else if (auto * data_uint32 = detail::getIndexesData(column)) - return detail::getUniqueIndexImpl(*data_uint32); - else if (auto * data_uint64 = detail::getIndexesData(column)) - return detail::getUniqueIndexImpl(*data_uint64); - else - throw Exception("Indexes column for makeSubindex must be ColumnUInt, got" + column.getName(), - ErrorCodes::LOGICAL_ERROR); + template const PaddedPODArray * getIndexesData(const IColumn & indexes); + template const PaddedPODArray * getIndexesData(const IColumn & indexes); + template const PaddedPODArray * getIndexesData(const IColumn & indexes); + template const PaddedPODArray * getIndexesData(const IColumn & indexes); } } diff --git a/dbms/src/Columns/ColumnsCommon.h b/dbms/src/Columns/ColumnsCommon.h index bc16af0b715..1902774529b 100644 --- a/dbms/src/Columns/ColumnsCommon.h +++ b/dbms/src/Columns/ColumnsCommon.h @@ -41,17 +41,17 @@ void filterArraysImplOnlyData( namespace detail { template - const PaddedPODArray * getIndexesData(const ColumnPtr & indexes); + const PaddedPODArray * getIndexesData(const IColumn & indexes); } /// Check limit <= indexes->size() and call column.indexImpl(const PaddedPodArray & indexes, size_t limit). template -ColumnPtr selectIndexImpl(const Column & column, const ColumnPtr & indexes, size_t limit) +ColumnPtr selectIndexImpl(const Column & column, const IColumn & indexes, size_t limit) { if (limit == 0) - limit = indexes->size(); + limit = indexes.size(); - if (indexes->size() < limit) + if (indexes.size() < limit) throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); if (auto * data_uint8 = detail::getIndexesData(indexes)) @@ -63,7 +63,7 @@ ColumnPtr selectIndexImpl(const Column & column, const ColumnPtr & indexes, size else if (auto * data_uint64 = detail::getIndexesData(indexes)) return column.template indexImpl(*data_uint64, limit); else - throw Exception("Indexes column for IColumn::select must be ColumnUInt, got" + indexes->getName(), + throw Exception("Indexes column for IColumn::select must be ColumnUInt, got" + indexes.getName(), ErrorCodes::LOGICAL_ERROR); } @@ -72,9 +72,4 @@ ColumnPtr selectIndexImpl(const Column & column, const ColumnPtr & indexes, size template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; - - -/// Get unique values from index column (ColumnUInt*). -MutableColumnPtr makeSubIndex(IColumn & column); - } diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 28fcbadee6f..5dcc6b25cfe 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -194,7 +194,7 @@ public: /// Creates new column with values column[indexes[:limit]]. If limit is 0, all indexes are used. /// Indexes must be one of the ColumnUInt. For default implementation, see selectIndexImpl from ColumnsCommon.h - virtual Ptr index(const Ptr & indexes, size_t limit) const = 0; + virtual Ptr index(const IColumn & indexes, size_t limit) const = 0; /** Compares (*this)[n] and rhs[m]. * Returns negative number, 0, or positive number (*this)[n] is less, equal, greater than rhs[m] respectively. diff --git a/dbms/src/Columns/IColumnDummy.h b/dbms/src/Columns/IColumnDummy.h index 7837d2b200e..b05382aa7ab 100644 --- a/dbms/src/Columns/IColumnDummy.h +++ b/dbms/src/Columns/IColumnDummy.h @@ -87,9 +87,9 @@ public: return cloneDummy(limit ? std::min(s, limit) : s); } - ColumnPtr index(const ColumnPtr & indexes, size_t limit) const override + ColumnPtr index(const IColumn & indexes, size_t limit) const override { - if (indexes->size() < limit) + if (indexes.size() < limit) throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); return cloneDummy(limit ? limit : s); diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index 9aba22d068e..15fe1ef6c81 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -7,9 +7,13 @@ namespace DB class IColumnUnique : public IColumn { public: + using ColumnUniquePtr = IColumn::template immutable_ptr; + using MutableColumnUniquePtr = IColumn::template mutable_ptr; + /// Column always contains Null if it's Nullable and empty string if it's String or Nullable(String). /// So, size may be greater than the number of inserted unique values. virtual const ColumnPtr & getNestedColumn() const = 0; + size_t size() const override { return getNestedColumn()->size(); } /// Appends new value at the end of column (column's size is increased by 1). @@ -19,7 +23,18 @@ public: virtual size_t uniqueInsertFrom(const IColumn & src, size_t n) = 0; /// Appends range of elements from other column. /// Could be used to concatenate columns. - virtual ColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; + virtual MutableColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; + + struct IndexesWithOverflow + { + MutableColumnPtr indexes; + MutableColumnPtr overflowed_keys; + }; + /// Like uniqueInsertRangeFrom, but doesn't insert keys if inner dictionary has more than max_dictionary_size keys. + /// Keys that won't be inserted into dictionary will be into overflowed_keys, indexes will be calculated for + /// concatenation of nested column (which can be got from getNestedColumn() function) and overflowed_keys. + virtual IndexesWithOverflow uniqueInsertRangeWithOverflow(const IColumn & src, size_t start, + size_t length, size_t max_dictionary_size) = 0; /// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented). /// Is used to optimize some computations (in aggregation, for example). @@ -33,7 +48,18 @@ public: virtual size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) = 0; -// virtual size_t getInsertionPoint(const char * pos, size_t length) const = 0; + /// Column which contains the set of necessary for serialization keys. Such that empty column after + /// uniqueInsertRangeFrom(column->cut(offset, limit), 0, limit) call will contain the same set of keys. + struct SerializableState + { + ColumnPtr column; + size_t offset; + size_t limit; + }; + + virtual SerializableState getSerializableState() const = 0; + +// virtual MutableColumnPtr getInsertionPoints(const ColumnPtr & keys) const = 0; // // virtual bool has(const char * pos, size_t length) const { return getInsertionPoint(pos, length) != size(); } @@ -74,7 +100,7 @@ public: throw Exception("Method deserializeAndInsertFromArena is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } - ColumnPtr index(const ColumnPtr &, size_t) const override + ColumnPtr index(const IColumn &, size_t) const override { throw Exception("Method index is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } @@ -110,4 +136,7 @@ public: } }; +using ColumnUniquePtr = IColumnUnique::ColumnUniquePtr; +using MutableColumnUniquePtr = IColumnUnique::MutableColumnUniquePtr; + } diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 836fb4d328c..86f95d3b372 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -59,9 +59,14 @@ NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) { - IDataType::InputStreamGetter input_stream_getter = [&] (const IDataType::SubstreamPath &) { return &istr; }; - auto state = type.createDeserializeBinaryBulkState(); - type.deserializeBinaryBulkWithMultipleStreams(column, input_stream_getter, rows, avg_value_size_hint, false, {}, state); + IDataType::DeserializeBinaryBulkSettings settings; + settings.getter = [&](IDataType::SubstreamPath) -> ReadBuffer * { return &istr; }; + settings.avg_value_size_hint = avg_value_size_hint; + settings.position_independent_encoding = false; + + IDataType::DeserializeBinaryBulkStatePtr state; + type.deserializeBinaryBulkStatePrefix(settings, state); + type.deserializeBinaryBulkWithMultipleStreams(column, rows, settings, state); if (column.size() != rows) throw Exception("Cannot read all data in NativeBlockInputStream.", ErrorCodes::CANNOT_READ_ALL_DATA); diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index b02d435b39f..cc361ceab8a 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -52,8 +52,15 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr else full_column = column; - IDataType::OutputStreamGetter output_stream_getter = [&] (const IDataType::SubstreamPath &) { return &ostr; }; - type.serializeBinaryBulkWithMultipleStreams(*full_column, output_stream_getter, offset, limit, false, {}); + IDataType::SerializeBinaryBulkSettings settings; + settings.getter = [&ostr](IDataType::SubstreamPath) -> WriteBuffer * { return &ostr; }; + settings.position_independent_encoding = false; + settings.max_dictionary_size = 0; + + IDataType::SerializeBinaryBulkStatePtr state; + type.serializeBinaryBulkStatePrefix(settings, state); + type.serializeBinaryBulkWithMultipleStreams(*full_column, offset, limit, settings, state); + type.serializeBinaryBulkStateSuffix(settings, state); } diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 247d3fc193f..d17c1def50d 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -144,37 +144,67 @@ namespace } -void DataTypeArray::enumerateStreams(StreamCallback callback, SubstreamPath path) const +void DataTypeArray::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::ArraySizes); callback(path); path.back() = Substream::ArrayElements; nested->enumerateStreams(callback, path); + path.pop_back(); +} + + +void DataTypeArray::serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::ArrayElements); + nested->serializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); +} + + +void DataTypeArray::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::ArrayElements); + nested->serializeBinaryBulkStateSuffix(settings, state); + settings.path.pop_back(); +} + + +void DataTypeArray::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::ArrayElements); + nested->deserializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); } void DataTypeArray::serializeBinaryBulkWithMultipleStreams( const IColumn & column, - OutputStreamGetter getter, size_t offset, size_t limit, - bool position_independent_encoding, - SubstreamPath path) const + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const { const ColumnArray & column_array = typeid_cast(column); /// First serialize array sizes. - path.push_back(Substream::ArraySizes); - if (auto stream = getter(path)) + settings.path.push_back(Substream::ArraySizes); + if (auto stream = settings.getter(settings.path)) { - if (position_independent_encoding) + if (settings.position_independent_encoding) serializeArraySizesPositionIndependent(column, *stream, offset, limit); else DataTypeNumber().serializeBinaryBulk(*column_array.getOffsetsPtr(), *stream, offset, limit); } /// Then serialize contents of arrays. - path.back() = Substream::ArrayElements; + settings.path.back() = Substream::ArrayElements; const ColumnArray::Offsets & offset_values = column_array.getOffsets(); if (offset > offset_values.size()) @@ -196,31 +226,29 @@ void DataTypeArray::serializeBinaryBulkWithMultipleStreams( : 0; if (limit == 0 || nested_limit) - nested->serializeBinaryBulkWithMultipleStreams(column_array.getData(), getter, nested_offset, nested_limit, position_independent_encoding, path); + nested->serializeBinaryBulkWithMultipleStreams(column_array.getData(), nested_offset, nested_limit, settings, state); + settings.path.pop_back(); } void DataTypeArray::deserializeBinaryBulkWithMultipleStreams( IColumn & column, - InputStreamGetter getter, size_t limit, - double /*avg_value_size_hint*/, - bool position_independent_encoding, - SubstreamPath path, - const DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const { ColumnArray & column_array = typeid_cast(column); - path.push_back(Substream::ArraySizes); - if (auto stream = getter(path)) + settings.path.push_back(Substream::ArraySizes); + if (auto stream = settings.getter(settings.path)) { - if (position_independent_encoding) + if (settings.position_independent_encoding) deserializeArraySizesPositionIndependent(column, *stream, limit); else DataTypeNumber().deserializeBinaryBulk(column_array.getOffsetsColumn(), *stream, limit, 0); } - path.back() = Substream::ArrayElements; + settings.path.back() = Substream::ArrayElements; ColumnArray::Offsets & offset_values = column_array.getOffsets(); IColumn & nested_column = column_array.getData(); @@ -230,7 +258,8 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreams( if (last_offset < nested_column.size()) throw Exception("Nested column is longer than last offset", ErrorCodes::LOGICAL_ERROR); size_t nested_limit = last_offset - nested_column.size(); - nested->deserializeBinaryBulkWithMultipleStreams(nested_column, getter, nested_limit, 0, position_independent_encoding, path, state); + nested->deserializeBinaryBulkWithMultipleStreams(nested_column, nested_limit, settings, state); + settings.path.pop_back(); /// Check consistency between offsets and elements subcolumns. /// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER. diff --git a/dbms/src/DataTypes/DataTypeArray.h b/dbms/src/DataTypes/DataTypeArray.h index 83df57f69d6..51a22ce84dd 100644 --- a/dbms/src/DataTypes/DataTypeArray.h +++ b/dbms/src/DataTypes/DataTypeArray.h @@ -61,29 +61,32 @@ public: * This is necessary, because when implementing nested structures, several arrays can have common sizes. */ - void enumerateStreams(StreamCallback callback, SubstreamPath path) const override; + void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + + void serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; void serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - OutputStreamGetter getter, - size_t offset, - size_t limit, - bool position_independent_encoding, - SubstreamPath path) const override; - - DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const override - { - return nested->createDeserializeBinaryBulkState(); - } + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - InputStreamGetter getter, - size_t limit, - double avg_value_size_hint, - bool position_independent_encoding, - SubstreamPath path, - const DeserializeBinaryBulkStatePtr & state) const override; + IColumn & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; MutableColumnPtr createColumn() const override; diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index 0f532cab258..9b4531fb5a0 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -37,54 +37,83 @@ bool DataTypeNullable::onlyNull() const } -void DataTypeNullable::enumerateStreams(StreamCallback callback, SubstreamPath path) const +void DataTypeNullable::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::NullMap); callback(path); path.back() = Substream::NullableElements; nested_data_type->enumerateStreams(callback, path); + path.pop_back(); +} + + +void DataTypeNullable::serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::NullableElements); + nested_data_type->serializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); +} + + +void DataTypeNullable::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::NullableElements); + nested_data_type->serializeBinaryBulkStateSuffix(settings, state); + settings.path.pop_back(); +} + + +void DataTypeNullable::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::NullableElements); + nested_data_type->deserializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); } void DataTypeNullable::serializeBinaryBulkWithMultipleStreams( const IColumn & column, - OutputStreamGetter getter, size_t offset, size_t limit, - bool position_independent_encoding, - SubstreamPath path) const + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const { const ColumnNullable & col = static_cast(column); col.checkConsistency(); /// First serialize null map. - path.push_back(Substream::NullMap); - if (auto stream = getter(path)) + settings.path.push_back(Substream::NullMap); + if (auto stream = settings.getter(settings.path)) DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), *stream, offset, limit); /// Then serialize contents of arrays. - path.back() = Substream::NullableElements; - nested_data_type->serializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, offset, limit, position_independent_encoding, path); + settings.path.back() = Substream::NullableElements; + nested_data_type->serializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), offset, limit, settings, state); + settings.path.pop_back(); } void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( IColumn & column, - InputStreamGetter getter, size_t limit, - double avg_value_size_hint, - bool position_independent_encoding, - SubstreamPath path, - const DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const { ColumnNullable & col = static_cast(column); - path.push_back(Substream::NullMap); - if (auto stream = getter(path)) + settings.path.push_back(Substream::NullMap); + if (auto stream = settings.getter(settings.path)) DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0); - path.back() = Substream::NullableElements; - nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, limit, avg_value_size_hint, position_independent_encoding, path, state); + settings.path.back() = Substream::NullableElements; + nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), limit, settings, state); + settings.path.pop_back(); } diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index d96692bed68..b8d4e5488d3 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -17,29 +17,32 @@ public: std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } - void enumerateStreams(StreamCallback callback, SubstreamPath path) const override; + void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + + void serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; void serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - OutputStreamGetter getter, - size_t offset, - size_t limit, - bool position_independent_encoding, - SubstreamPath path) const override; - - DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const override - { - return nested_data_type->createDeserializeBinaryBulkState(); - } + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - InputStreamGetter getter, - size_t limit, - double avg_value_size_hint, - bool position_independent_encoding, - SubstreamPath path, - const DeserializeBinaryBulkStatePtr & state) const override; + IColumn & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; void serializeBinary(const Field & field, WriteBuffer & ostr) const override { nested_data_type->serializeBinary(field, ostr); } void deserializeBinary(Field & field, ReadBuffer & istr) const override { nested_data_type->deserializeBinary(field, istr); } diff --git a/dbms/src/DataTypes/DataTypeTuple.cpp b/dbms/src/DataTypes/DataTypeTuple.cpp index f66ad76c85d..47421415baf 100644 --- a/dbms/src/DataTypes/DataTypeTuple.cpp +++ b/dbms/src/DataTypes/DataTypeTuple.cpp @@ -282,7 +282,7 @@ void DataTypeTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons }); } -void DataTypeTuple::enumerateStreams(StreamCallback callback, SubstreamPath path) const +void DataTypeTuple::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::TupleElement); for (const auto i : ext::range(0, ext::size(elems))) @@ -290,44 +290,133 @@ void DataTypeTuple::enumerateStreams(StreamCallback callback, SubstreamPath path path.back().tuple_element_name = names[i]; elems[i]->enumerateStreams(callback, path); } + path.pop_back(); +} + +struct SerializeBinaryBulkStateTuple : public IDataType::SerializeBinaryBulkState +{ + std::vector states; +}; + +struct DeserializeBinaryBulkStateTuple : public IDataType::DeserializeBinaryBulkState +{ + std::vector states; +}; + +static SerializeBinaryBulkStateTuple * checkAndGetTupleSerializeState(IDataType::SerializeBinaryBulkStatePtr & state) +{ + if (!state) + throw Exception("Got empty state for DataTypeTuple.", ErrorCodes::LOGICAL_ERROR); + + auto * tuple_state = typeid_cast(state.get()); + if (!tuple_state) + throw Exception("Invalid SerializeBinaryBulkState for DataTypeTuple. Expected: " + + demangle(typeid(SerializeBinaryBulkStateTuple).name()) + ", got " + + demangle(typeid(*state).name()), ErrorCodes::LOGICAL_ERROR); + + return tuple_state; +} + +static DeserializeBinaryBulkStateTuple * checkAndGetTupleDeserializeState(IDataType::DeserializeBinaryBulkStatePtr & state) +{ + if (!state) + throw Exception("Got empty state for DataTypeTuple.", ErrorCodes::LOGICAL_ERROR); + + auto * tuple_state = typeid_cast(state.get()); + if (!tuple_state) + throw Exception("Invalid DeserializeBinaryBulkState for DataTypeTuple. Expected: " + + demangle(typeid(DeserializeBinaryBulkStateTuple).name()) + ", got " + + demangle(typeid(*state).name()), ErrorCodes::LOGICAL_ERROR); + + return tuple_state; +} + +void DataTypeTuple::serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + auto tuple_state = std::make_shared(); + tuple_state->states.resize(elems.size()); + + settings.path.push_back(Substream::TupleElement); + for (size_t i = 0; i < elems.size(); ++i) + { + settings.path.back().tuple_element_name = names[i]; + elems[i]->serializeBinaryBulkStatePrefix(settings, tuple_state->states[i]); + } + settings.path.pop_back(); + + state = std::move(tuple_state); +} + +void DataTypeTuple::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + auto * tuple_state = checkAndGetTupleSerializeState(state); + + settings.path.push_back(Substream::TupleElement); + for (size_t i = 0; i < elems.size(); ++i) + { + settings.path.back().tuple_element_name = names[i]; + elems[i]->serializeBinaryBulkStateSuffix(settings, tuple_state->states[i]); + } + settings.path.pop_back(); +} + +void DataTypeTuple::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + auto tuple_state = std::make_shared(); + tuple_state->states.resize(elems.size()); + + settings.path.push_back(Substream::TupleElement); + for (size_t i = 0; i < elems.size(); ++i) + { + settings.path.back().tuple_element_name = names[i]; + elems[i]->deserializeBinaryBulkStatePrefix(settings, tuple_state->states[i]); + } + settings.path.pop_back(); + + state = std::move(tuple_state); } void DataTypeTuple::serializeBinaryBulkWithMultipleStreams( const IColumn & column, - OutputStreamGetter getter, size_t offset, size_t limit, - bool position_independent_encoding, - SubstreamPath path) const + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const { - path.push_back(Substream::TupleElement); + auto * tuple_state = checkAndGetTupleSerializeState(state); + + settings.path.push_back(Substream::TupleElement); for (const auto i : ext::range(0, ext::size(elems))) { - path.back().tuple_element_name = names[i]; - elems[i]->serializeBinaryBulkWithMultipleStreams( - extractElementColumn(column, i), getter, offset, limit, position_independent_encoding, path); + settings.path.back().tuple_element_name = names[i]; + auto & element_col = extractElementColumn(column, i); + elems[i]->serializeBinaryBulkWithMultipleStreams(element_col, offset, limit, settings, tuple_state->states[i]); } + settings.path.pop_back(); } void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams( IColumn & column, - InputStreamGetter getter, size_t limit, - double avg_value_size_hint, - bool position_independent_encoding, - SubstreamPath path, - const DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const { - auto * tuple_state = typeid_cast(state.get()); + auto * tuple_state = checkAndGetTupleDeserializeState(state); - path.push_back(Substream::TupleElement); + settings.path.push_back(Substream::TupleElement); for (const auto i : ext::range(0, ext::size(elems))) { - path.back().tuple_element_name = names[i]; - elems[i]->deserializeBinaryBulkWithMultipleStreams( - extractElementColumn(column, i), getter, limit, avg_value_size_hint, - position_independent_encoding, path, tuple_state->states[i]); + settings.path.back().tuple_element_name = names[i]; + auto & element_col = extractElementColumn(column, i); + elems[i]->deserializeBinaryBulkWithMultipleStreams(element_col, limit, settings, tuple_state->states[i]); } + settings.path.pop_back(); } MutableColumnPtr DataTypeTuple::createColumn() const diff --git a/dbms/src/DataTypes/DataTypeTuple.h b/dbms/src/DataTypes/DataTypeTuple.h index aeabd26f5a2..e95b349a041 100644 --- a/dbms/src/DataTypes/DataTypeTuple.h +++ b/dbms/src/DataTypes/DataTypeTuple.h @@ -53,40 +53,32 @@ public: /** Each sub-column in a tuple is serialized in separate stream. */ - void enumerateStreams(StreamCallback callback, SubstreamPath path) const override; + void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + + void serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; void serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - OutputStreamGetter getter, - size_t offset, - size_t limit, - bool position_independent_encoding, - SubstreamPath path) const override; - - struct DeserializeBinaryBulkStateTuple : public IDataType::DeserializeBinaryBulkState - { - std::vector states; - DeserializeBinaryBulkStateTuple(const DataTypes & types) - { - states.reserve(types.size()); - for (auto & type : types) - states.emplace_back(type->createDeserializeBinaryBulkState()); - } - }; - - DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const override - { - return std::make_shared(elems); - } + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - InputStreamGetter getter, - size_t limit, - double avg_value_size_hint, - bool position_independent_encoding, - SubstreamPath path, - const DeserializeBinaryBulkStatePtr & state) const override; + IColumn & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; MutableColumnPtr createColumn() const override; diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 9ec91eb4a68..0ec0c355e9f 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -52,134 +52,543 @@ DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_, Dat + dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } -void DataTypeWithDictionary::enumerateStreams(StreamCallback callback, SubstreamPath path) const +void DataTypeWithDictionary::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::DictionaryKeys); dictionary_type->enumerateStreams(callback, path); path.back() = Substream::DictionaryIndexes; indexes_type->enumerateStreams(callback, path); + path.pop_back(); +} + +struct KeysSerializationVersion +{ + /// Write keys as full column. No indexes is written. Structure: + /// .dict.bin : [version - 32 bits][keys] + /// .dict.mrk : [marks for keys] + // FullColumn = 0, + /// Write all keys in serializePostfix and read in deserializePrefix. + /// .dict.bin : [version - 32 bits][indexes type - 32 bits][keys] + /// .bin : [indexes] + /// .mrk : [marks for indexes] + // SingleDictionary, + /// Write distinct set of keys for each granule. Structure: + /// .dict.bin : [version - 32 bits][indexes type - 32 bits][keys] + /// .dict.mrk : [marks for keys] + /// .bin : [indexes] + /// .mrk : [marks for indexes] + // DictionaryPerGranule, + + enum Value + { + SingleDictionaryWithAdditionalKeysPerBlock = 1, + }; + + Value value; + + static void checkVersion(UInt64 version) + { + if (version != SingleDictionaryWithAdditionalKeysPerBlock) + throw Exception("Invalid version for DataTypeWithDictionary key column.", ErrorCodes::LOGICAL_ERROR); + } + + KeysSerializationVersion(UInt64 version) : value(static_cast(version)) { checkVersion(version); } +}; + +struct IndexesSerializationType +{ + using SerializationType = UInt64; + static constexpr UInt64 NeedGlobalDictionaryBit = 1u << 8u; + static constexpr UInt64 HasAdditionalKeysBit = 1u << 9u; + + enum Type + { + TUInt8 = 0, + TUInt16, + TUInt32, + TUInt64, + }; + + Type type; + bool has_additional_keys; + bool need_global_dictionary; + + static constexpr SerializationType resetFlags(SerializationType type) + { + return type & (~(HasAdditionalKeysBit | NeedGlobalDictionaryBit)); + } + + static void checkType(SerializationType type) + { + UInt64 value = resetFlags(type); + if (value <= TUInt64) + return; + + throw Exception("Invalid type for DataTypeWithDictionary index column.", ErrorCodes::LOGICAL_ERROR); + } + + void serialize(WriteBuffer & buffer) const + { + SerializationType val = type; + if (has_additional_keys) + val |= HasAdditionalKeysBit; + if (need_global_dictionary) + val |= NeedGlobalDictionaryBit; + writeIntBinary(val, buffer); + } + + void deserialize(ReadBuffer & buffer) + { + SerializationType val; + readIntBinary(val, buffer); + checkType(val); + has_additional_keys = (val & HasAdditionalKeysBit) != 0; + need_global_dictionary = (val & NeedGlobalDictionaryBit) != 0; + type = static_cast(resetFlags(val)); + } + + IndexesSerializationType(const IDataType & data_type, bool has_additional_keys, bool need_global_dictionary) + : has_additional_keys(has_additional_keys), need_global_dictionary(need_global_dictionary) + { + if (typeid_cast(&data_type)) + type = TUInt8; + else if (typeid_cast(&data_type)) + type = TUInt16; + else if (typeid_cast(&data_type)) + type = TUInt32; + else if (typeid_cast(&data_type)) + type = TUInt64; + else + throw Exception("Invalid DataType for IndexesSerializationType. Expected UInt*, got " + data_type.getName(), + ErrorCodes::LOGICAL_ERROR); + } + + DataTypePtr getDataType() const + { + if (type == TUInt8) + return std::make_shared(); + if (type == TUInt16) + return std::make_shared(); + if (type == TUInt32) + return std::make_shared(); + if (type == TUInt64) + return std::make_shared(); + + throw Exception("Can't create DataType from IndexesSerializationType.", ErrorCodes::LOGICAL_ERROR); + } + + IndexesSerializationType() = default; +}; + +struct SerializeStateWithDictionary : public IDataType::SerializeBinaryBulkState +{ + KeysSerializationVersion key_version; + MutableColumnUniquePtr global_dictionary; + + explicit SerializeStateWithDictionary( + UInt64 key_version, + MutableColumnUniquePtr && column_unique) + : key_version(key_version) + , global_dictionary(std::move(column_unique)) {} +}; + +struct DeserializeStateWithDictionary : public IDataType::DeserializeBinaryBulkState +{ + KeysSerializationVersion key_version; + ColumnUniquePtr global_dictionary; + UInt64 num_bytes_in_dictionary; + + IndexesSerializationType index_type; + MutableColumnPtr additional_keys; + UInt64 num_pending_rows = 0; + + explicit DeserializeStateWithDictionary(UInt64 key_version) : key_version(key_version) {} +}; + +static SerializeStateWithDictionary * checkAndGetWithDictionarySerializeState( + IDataType::SerializeBinaryBulkStatePtr & state) +{ + if (!state) + throw Exception("Got empty state for DataTypeWithDictionary.", ErrorCodes::LOGICAL_ERROR); + + auto * with_dictionary_state = typeid_cast(state.get()); + if (!with_dictionary_state) + throw Exception("Invalid SerializeBinaryBulkState for DataTypeWithDictionary. Expected: " + + demangle(typeid(SerializeStateWithDictionary).name()) + ", got " + + demangle(typeid(*state).name()), ErrorCodes::LOGICAL_ERROR); + + return with_dictionary_state; +} + +static DeserializeStateWithDictionary * checkAndGetWithDictionaryDeserializeState( + IDataType::DeserializeBinaryBulkStatePtr & state) +{ + if (!state) + throw Exception("Got empty state for DataTypeWithDictionary.", ErrorCodes::LOGICAL_ERROR); + + auto * with_dictionary_state = typeid_cast(state.get()); + if (!with_dictionary_state) + throw Exception("Invalid DeserializeBinaryBulkState for DataTypeWithDictionary. Expected: " + + demangle(typeid(DeserializeStateWithDictionary).name()) + ", got " + + demangle(typeid(*state).name()), ErrorCodes::LOGICAL_ERROR); + + return with_dictionary_state; +} + +void DataTypeWithDictionary::serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::DictionaryKeys); + auto * stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!stream) + throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStatePrefix", + ErrorCodes::LOGICAL_ERROR); + + /// Write version and create SerializeBinaryBulkState. + UInt64 key_version = KeysSerializationVersion::SingleDictionaryWithAdditionalKeysPerBlock; + + writeIntBinary(key_version, *stream); + + auto column_unique = createColumnUnique(*dictionary_type, *indexes_type); + state = std::make_shared(key_version, std::move(column_unique)); +} + +void DataTypeWithDictionary::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + auto * state_with_dictionary = checkAndGetWithDictionarySerializeState(state); + KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value); + + if (state_with_dictionary->global_dictionary) + { + auto unique_state = state_with_dictionary->global_dictionary->getSerializableState(); + UInt64 num_keys = unique_state.limit; + if (settings.max_dictionary_size) + { + settings.path.push_back(Substream::DictionaryKeys); + auto * stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!stream) + throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStateSuffix", + ErrorCodes::LOGICAL_ERROR); + + writeIntBinary(num_keys, *stream); + removeNullable(dictionary_type)->serializeBinaryBulk(*unique_state.column, *stream, + unique_state.offset, unique_state.limit); + } + } +} + +void DataTypeWithDictionary::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::DictionaryKeys); + auto * stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!stream) + throw Exception("Got empty stream in DataTypeWithDictionary::deserializeBinaryBulkStatePrefix", + ErrorCodes::LOGICAL_ERROR); + + UInt64 keys_version; + readIntBinary(keys_version, *stream); + + state = std::make_shared(keys_version); +} + +namespace +{ + template + PaddedPODArray * getIndexesData(IColumn & indexes) + { + auto * column = typeid_cast *>(&indexes); + if (column) + return &column->getData(); + + return nullptr; + } + + template + MutableColumnPtr mapUniqueIndexImpl(PaddedPODArray & index) + { + HashMap hash_map; + for (auto val : index) + hash_map.insert({val, hash_map.size()}); + + auto res_col = ColumnVector::create(); + auto & data = res_col->getData(); + + data.resize(hash_map.size()); + for (auto val : hash_map) + data[val.second] = val.first; + + for (auto & ind : index) + ind = hash_map[ind]; + + return std::move(res_col); + } + + /// Returns unique values of column. Write new index to column. + MutableColumnPtr mapUniqueIndex(IColumn & column) + { + if (auto * data_uint8 = getIndexesData(column)) + return mapUniqueIndexImpl(*data_uint8); + else if (auto * data_uint16 = getIndexesData(column)) + return mapUniqueIndexImpl(*data_uint16); + else if (auto * data_uint32 = getIndexesData(column)) + return mapUniqueIndexImpl(*data_uint32); + else if (auto * data_uint64 = getIndexesData(column)) + return mapUniqueIndexImpl(*data_uint64); + else + throw Exception("Indexes column for getUniqueIndex must be ColumnUInt, got" + column.getName(), + ErrorCodes::LOGICAL_ERROR); + } + + template + MutableColumnPtr mapIndexWithOverflow(PaddedPODArray & index, size_t max_val) + { + HashMap hash_map; + HashMap hash_map_with_overflow; + + for (auto val : index) + { + auto & map = val < max_val ? hash_map : hash_map_with_overflow; + map.insert({val, map.size()}); + } + + auto index_map_col = ColumnVector::create(); + auto & index_data = index_map_col->getData(); + + index_data.resize(hash_map.size()); + for (auto val : hash_map) + index_data[val.second] = val.first; + + for (auto & val : index) + val = val < max_val ? hash_map[val] + : hash_map_with_overflow[val] + hash_map.size(); + + return index_map_col; + } + + MutableColumnPtr mapIndexWithOverflow(IColumn & column, size_t max_size) + { + if (auto * data_uint8 = getIndexesData(column)) + return mapIndexWithOverflow(*data_uint8, max_size); + else if (auto * data_uint16 = getIndexesData(column)) + return mapIndexWithOverflow(*data_uint16, max_size); + else if (auto * data_uint32 = getIndexesData(column)) + return mapIndexWithOverflow(*data_uint32, max_size); + else if (auto * data_uint64 = getIndexesData(column)) + return mapIndexWithOverflow(*data_uint64, max_size); + else + throw Exception("Indexes column for makeIndexWithOverflow must be ColumnUInt, got" + column.getName(), + ErrorCodes::LOGICAL_ERROR); + } } void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - OutputStreamGetter getter, - size_t offset, - size_t limit, - bool position_independent_encoding, - SubstreamPath path) const + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const { + settings.path.push_back(Substream::DictionaryKeys); + auto * keys_stream = settings.getter(settings.path); + settings.path.back() = Substream::DictionaryIndexes; + auto * indexes_stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!keys_stream && !indexes_stream) + return; + + if (!keys_stream) + throw Exception("Got empty stream for DataTypeWithDictionary keys.", ErrorCodes::LOGICAL_ERROR); + + if (!indexes_stream) + throw Exception("Got empty stream for DataTypeWithDictionary indexes.", ErrorCodes::LOGICAL_ERROR); + const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + auto * state_with_dictionary = checkAndGetWithDictionarySerializeState(state); + auto & global_dictionary = state_with_dictionary->global_dictionary; + KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value); + + auto unique_state = global_dictionary->getSerializableState(); + bool was_global_dictionary_written = unique_state.limit >= settings.max_dictionary_size; + + const auto & indexes = column_with_dictionary.getIndexesPtr(); + const auto & keys = column_with_dictionary.getUnique()->getSerializableState().column; + size_t max_limit = column.size() - offset; limit = limit ? std::min(limit, max_limit) : max_limit; - path.push_back(Substream::DictionaryIndexes); - if (auto stream = getter(path)) + /// Create pair (used_keys, sub_index) which is the dictionary for [offset, offset + limit) range. + MutableColumnPtr sub_index = (*indexes->cut(offset, limit)).mutate(); + auto unique_indexes = mapUniqueIndex(*sub_index); + /// unique_indexes->index(*sub_index) == indexes[offset:offset + limit] + MutableColumnPtr used_keys = (*keys->index(*unique_indexes, 0)).mutate(); + + if (settings.max_dictionary_size) { - const auto & indexes = column_with_dictionary.getIndexesPtr(); - const auto & keys = column_with_dictionary.getUnique()->getNestedColumn(); - MutableColumnPtr sub_index = (*indexes->cut(offset, limit)).mutate(); - ColumnPtr unique_indexes = makeSubIndex(*sub_index); - /// unique_indexes->index(sub_index) == indexes[offset:offset + limit] - auto used_keys = keys->index(unique_indexes, 0); - /// (used_keys, sub_index) is ColumnWithDictionary for range [offset:offset + limit] - - UInt64 used_keys_size = used_keys->size(); - writeIntBinary(used_keys_size, *stream); - - UInt64 indexes_size = sub_index->size(); - writeIntBinary(indexes_size, *stream); - - path.back() = Substream::DictionaryKeys; - dictionary_type->serializeBinaryBulkWithMultipleStreams(*used_keys, getter, 0, 0, - position_independent_encoding, path); - - indexes_type->serializeBinaryBulk(*sub_index, *stream, 0, limit); + /// Insert used_keys into global dictionary and update sub_index. + auto indexes_with_overflow = global_dictionary->uniqueInsertRangeWithOverflow(*used_keys, 0, used_keys->size(), + settings.max_dictionary_size); + sub_index = (*indexes_with_overflow.indexes->index(*sub_index, 0)).mutate(); + used_keys = std::move(indexes_with_overflow.overflowed_keys); } -} -struct DeserializeBinaryBulkStateWithDictionary : public IDataType::DeserializeBinaryBulkState -{ - UInt64 num_rows_to_read_until_next_index = 0; - ColumnPtr index; - IDataType::DeserializeBinaryBulkStatePtr state; + bool need_additional_keys = !used_keys->empty(); + bool need_dictionary = settings.max_dictionary_size != 0; + bool need_write_dictionary = !was_global_dictionary_written && unique_state.limit >= settings.max_dictionary_size; - explicit DeserializeBinaryBulkStateWithDictionary(IDataType::DeserializeBinaryBulkStatePtr && state) - : state(std::move(state)) {} -}; + IndexesSerializationType index_version(*indexes_type, need_additional_keys, need_dictionary); + index_version.serialize(*indexes_stream); -IDataType::DeserializeBinaryBulkStatePtr DataTypeWithDictionary::createDeserializeBinaryBulkState() const -{ - return std::make_shared( - dictionary_type->createDeserializeBinaryBulkState()); + unique_state = global_dictionary->getSerializableState(); + + if (need_write_dictionary) + { + /// Write global dictionary if it wasn't written and has too many keys. + UInt64 num_keys = unique_state.limit; + writeIntBinary(num_keys, *keys_stream); + removeNullable(dictionary_type)->serializeBinaryBulk(*unique_state.column, *keys_stream, unique_state.offset, num_keys); + } + + if (need_additional_keys) + { + UInt64 num_keys = used_keys->size(); + writeIntBinary(num_keys, *indexes_stream); + removeNullable(dictionary_type)->serializeBinaryBulk(*used_keys, *indexes_stream, 0, num_keys); + } + + UInt64 num_rows = sub_index->size(); + writeIntBinary(num_rows, *indexes_stream); + indexes_type->serializeBinaryBulk(*sub_index, *indexes_stream, 0, num_rows); } void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - InputStreamGetter getter, - size_t limit, - double /*avg_value_size_hint*/, - bool position_independent_encoding, - SubstreamPath path, - const DeserializeBinaryBulkStatePtr & state) const + IColumn & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const { ColumnWithDictionary & column_with_dictionary = typeid_cast(column); - auto dict_state = typeid_cast(state.get()); - if (dict_state == nullptr) - throw Exception("Invalid DeserializeBinaryBulkState.", ErrorCodes::LOGICAL_ERROR); + auto * state_with_dictionary = checkAndGetWithDictionaryDeserializeState(state); + KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value); - auto readIndexes = [&](ReadBuffer * stream, const ColumnPtr & index, size_t num_rows) + settings.path.push_back(Substream::DictionaryKeys); + auto * keys_stream = settings.getter(settings.path); + settings.path.back() = Substream::DictionaryIndexes; + auto * indexes_stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!keys_stream && !indexes_stream) + return; + + if (!keys_stream) + throw Exception("Got empty stream for DataTypeWithDictionary keys.", ErrorCodes::LOGICAL_ERROR); + + if (!indexes_stream) + throw Exception("Got empty stream for DataTypeWithDictionary indexes.", ErrorCodes::LOGICAL_ERROR); + + auto readDictionary = [this, state_with_dictionary, keys_stream, &column_with_dictionary]() { - auto index_col = indexes_type->createColumn(); - indexes_type->deserializeBinaryBulk(*index_col, *stream, num_rows, 0); - column_with_dictionary.getIndexes()->insertRangeFrom(*index->index(std::move(index_col), 0), 0, num_rows); + UInt64 num_keys; + readIntBinary(num_keys, *keys_stream); + + auto keys_type = removeNullable(dictionary_type); + auto global_dict_keys = keys_type->createColumn(); + keys_type->deserializeBinaryBulk(*global_dict_keys, *keys_stream, num_keys, 0); + + auto column_unique = createColumnUnique(*dictionary_type, *indexes_type); + column_unique->uniqueInsertRangeFrom(*global_dict_keys, 0, num_keys); + state_with_dictionary->global_dictionary = std::move(column_unique); }; - using CachedStreams = std::unordered_map; - CachedStreams cached_streams; - - IDataType::InputStreamGetter cached_stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer * + auto readAdditionalKeys = [this, state_with_dictionary, indexes_stream]() { - std::string stream_name = IDataType::getFileNameForStream("", path); - auto iter = cached_streams.find(stream_name); - if (iter == cached_streams.end()) - iter = cached_streams.insert({stream_name, getter(path)}).first; - return iter->second; + UInt64 num_keys; + readIntBinary(num_keys, *indexes_stream); + auto keys_type = removeNullable(dictionary_type); + state_with_dictionary->additional_keys = keys_type->createColumn(); + keys_type->deserializeBinaryBulk(*state_with_dictionary->additional_keys, *indexes_stream, num_keys, 0); }; - auto readDict = [&](UInt64 num_keys) + auto readIndexes = [this, state_with_dictionary, indexes_stream, &column_with_dictionary](UInt64 num_rows, + bool need_dictionary) { - auto dict_column = dictionary_type->createColumn(); - dictionary_type->deserializeBinaryBulkWithMultipleStreams(*dict_column, cached_stream_getter, num_keys, 0, - position_independent_encoding, path, dict_state->state); - return column_with_dictionary.getUnique()->uniqueInsertRangeFrom(*dict_column, 0, num_keys); - }; + MutableColumnPtr indexes_column = indexes_type->createColumn(); + indexes_type->deserializeBinaryBulk(*indexes_column, *indexes_stream, num_rows, 0); - path.push_back(Substream::DictionaryIndexes); + auto & global_dictionary = state_with_dictionary->global_dictionary; + const auto & additional_keys = state_with_dictionary->additional_keys; + auto * column_unique = column_with_dictionary.getUnique(); - if (auto stream = getter(path)) - { - path.back() = Substream::DictionaryKeys; + bool has_additional_keys = state_with_dictionary->additional_keys != nullptr; + bool column_is_empty = column_with_dictionary.empty(); + bool column_with_global_dictionary = column_unique == global_dictionary.get(); - while (limit) + if (!has_additional_keys && (column_is_empty || column_with_global_dictionary)) { - if (dict_state->num_rows_to_read_until_next_index == 0) - { - if (stream->eof()) - break; + if (column_is_empty) + column_with_dictionary.setUnique(global_dictionary); - UInt64 num_keys; - readIntBinary(num_keys, *stream); - readIntBinary(dict_state->num_rows_to_read_until_next_index, *stream); - dict_state->index = readDict(num_keys); + column_with_dictionary.getIndexes()->insertRangeFrom(*indexes_column, 0, num_rows); + } + else if (!need_dictionary) + { + auto indexes = column_unique->uniqueInsertRangeFrom(*additional_keys, 0, additional_keys->size()); + column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(*indexes_column, 0), 0, num_rows); + } + else + { + auto index_map = mapIndexWithOverflow(*indexes_column, global_dictionary->size()); + auto used_keys = global_dictionary->getNestedColumn()->index(*index_map, 0); + auto indexes = column_unique->uniqueInsertRangeFrom(*used_keys, 0, used_keys->size()); + + if (additional_keys) + { + size_t num_keys = additional_keys->size(); + auto additional_indexes = column_unique->uniqueInsertRangeFrom(*additional_keys, 0, num_keys); + indexes->insertRangeFrom(*additional_indexes, 0, num_keys); } - size_t num_rows_to_read = std::min(limit, dict_state->num_rows_to_read_until_next_index); - readIndexes(stream, dict_state->index, num_rows_to_read); - limit -= num_rows_to_read; - dict_state->num_rows_to_read_until_next_index -= num_rows_to_read; + column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(*indexes_column, 0), 0, num_rows); } + }; + + while (limit) + { + if (state_with_dictionary->num_pending_rows == 0) + { + if (indexes_stream->eof()) + break; + + state_with_dictionary->index_type.deserialize(*indexes_stream); + + if (state_with_dictionary->index_type.need_global_dictionary && !state_with_dictionary->global_dictionary) + readDictionary(); + + if (state_with_dictionary->index_type.has_additional_keys) + readAdditionalKeys(); + else + state_with_dictionary->additional_keys = nullptr; + + readIntBinary(state_with_dictionary->num_pending_rows, *indexes_stream); + } + + size_t num_rows_to_read = std::min(limit, state_with_dictionary->num_pending_rows); + readIndexes(num_rows_to_read, state_with_dictionary->index_type.need_global_dictionary); + limit -= num_rows_to_read; + state_with_dictionary->num_pending_rows -= num_rows_to_read; } } @@ -216,64 +625,69 @@ void DataTypeWithDictionary::deserializeImpl( } template -MutableColumnPtr DataTypeWithDictionary::createColumnImpl() const +MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDataType & keys_type) { - return ColumnWithDictionary::create(ColumnUnique::create(dictionary_type), - indexes_type->createColumn()); + return ColumnUnique::create(keys_type); } template -MutableColumnPtr DataTypeWithDictionary::createColumnImpl() const +MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDataType & keys_type, + const IDataType & indexes_type) { - if (typeid_cast(indexes_type.get())) - return createColumnImpl(); - if (typeid_cast(indexes_type.get())) - return createColumnImpl(); - if (typeid_cast(indexes_type.get())) - return createColumnImpl(); - if (typeid_cast(indexes_type.get())) - return createColumnImpl(); + if (typeid_cast(&indexes_type)) + return createColumnUniqueImpl(keys_type); + if (typeid_cast(&indexes_type)) + return createColumnUniqueImpl(keys_type); + if (typeid_cast(&indexes_type)) + return createColumnUniqueImpl(keys_type); + if (typeid_cast(&indexes_type)) + return createColumnUniqueImpl(keys_type); - throw Exception("The type of indexes must be unsigned integer, but got " + dictionary_type->getName(), + throw Exception("The type of indexes must be unsigned integer, but got " + indexes_type.getName(), ErrorCodes::LOGICAL_ERROR); } struct CreateColumnVector { - MutableColumnPtr & column; - const DataTypeWithDictionary * data_type_with_dictionary; - const IDataType * type; + MutableColumnUniquePtr & column; + const IDataType & keys_type; + const IDataType & indexes_type; + const IDataType * nested_type; - CreateColumnVector(MutableColumnPtr & column, const DataTypeWithDictionary * data_type_with_dictionary, - const IDataType * type) - : column(column), data_type_with_dictionary(data_type_with_dictionary), type(type) {} + CreateColumnVector(MutableColumnUniquePtr & column, const IDataType & keys_type, const IDataType & indexes_type) + : column(column), keys_type(keys_type), indexes_type(indexes_type), nested_type(&keys_type) + { + if (auto nullable_type = typeid_cast(&keys_type)) + nested_type = nullable_type->getNestedType().get(); + } template void operator()() { - if (typeid_cast *>(type)) - column = data_type_with_dictionary->createColumnImpl>(); + if (typeid_cast *>(nested_type)) + column = DataTypeWithDictionary::createColumnUniqueImpl>(keys_type, indexes_type); } }; -MutableColumnPtr DataTypeWithDictionary::createColumn() const +MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type, + const IDataType & indexes_type) { - auto type = dictionary_type; + auto * type = &keys_type; if (type->isNullable()) - type = static_cast(*dictionary_type).getNestedType(); + type = static_cast(keys_type).getNestedType().get(); if (type->isString()) - return createColumnImpl(); + return createColumnUniqueImpl(keys_type, indexes_type); if (type->isFixedString()) - return createColumnImpl(); - if (typeid_cast(type.get())) - return createColumnImpl>(); - if (typeid_cast(type.get())) - return createColumnImpl>(); + return createColumnUniqueImpl(keys_type, indexes_type); + if (typeid_cast(type)) + return createColumnUniqueImpl>(keys_type, indexes_type); + if (typeid_cast(type)) + return createColumnUniqueImpl>(keys_type, indexes_type); if (type->isNumber()) { - MutableColumnPtr column; - TypeListNumbers::forEach(CreateColumnVector(column, this, type.get())); + MutableColumnUniquePtr column; + TypeListNumbers::forEach(CreateColumnVector(column, keys_type, indexes_type)); if (!column) throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR); @@ -285,6 +699,13 @@ MutableColumnPtr DataTypeWithDictionary::createColumn() const ErrorCodes::LOGICAL_ERROR); } +MutableColumnPtr DataTypeWithDictionary::createColumn() const +{ + MutableColumnPtr indexes = indexes_type->createColumn(); + MutableColumnPtr dictionary = createColumnUnique(*dictionary_type, *indexes_type); + return ColumnWithDictionary::create(std::move(dictionary), std::move(indexes)); +} + bool DataTypeWithDictionary::equals(const IDataType & rhs) const { if (typeid(rhs) != typeid(*this)) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index 8492a1c8745..05e646f173d 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { @@ -22,26 +23,32 @@ public: } const char * getFamilyName() const override { return "WithDictionary"; } - void enumerateStreams(StreamCallback callback, SubstreamPath path) const override; + void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + + void serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, - OutputStreamGetter getter, size_t offset, size_t limit, - bool position_independent_encoding, - SubstreamPath path) const override; - - DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const override; + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; void deserializeBinaryBulkWithMultipleStreams( IColumn & column, - InputStreamGetter getter, size_t limit, - double avg_value_size_hint, - bool position_independent_encoding, - SubstreamPath path, - const DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void deserializeBinary(Field & field, ReadBuffer & istr) const override; @@ -139,6 +146,8 @@ public: bool onlyNull() const override { return false; } bool withDictionary() const override { return true; } + static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, const IDataType & indexes_type); + private: template @@ -156,10 +165,11 @@ private: DeserealizeFunctionPtr func, Args ... args) const; template - MutableColumnPtr createColumnImpl() const; + static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type); template - MutableColumnPtr createColumnImpl() const; + static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const IDataType & indexes_type); + friend struct CreateColumnVector; }; diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 1751ec9a3b1..7b3df678858 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -94,14 +94,62 @@ public: using SubstreamPath = std::vector; using StreamCallback = std::function; - virtual void enumerateStreams(StreamCallback callback, SubstreamPath path) const + virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { callback(path); } + void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); } + void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); } using OutputStreamGetter = std::function; using InputStreamGetter = std::function; + struct SerializeBinaryBulkState + { + virtual ~SerializeBinaryBulkState() = default; + }; + struct DeserializeBinaryBulkState + { + virtual ~DeserializeBinaryBulkState() = default; + }; + + using SerializeBinaryBulkStatePtr = std::shared_ptr; + using DeserializeBinaryBulkStatePtr = std::shared_ptr; + + struct SerializeBinaryBulkSettings + { + OutputStreamGetter getter; + SubstreamPath path; + + bool position_independent_encoding = true; + size_t max_dictionary_size = 0; + }; + + struct DeserializeBinaryBulkSettings + { + InputStreamGetter getter; + SubstreamPath path; + + bool position_independent_encoding = true; + /// If not zero, may be used to avoid reallocations while reading column of String type. + double avg_value_size_hint = 0; + }; + + /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. + virtual void serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & /*settings*/, + SerializeBinaryBulkStatePtr & /*state*/) const {} + + /// Call after serializeBinaryBulkWithMultipleStreams chain to finish serialization. + virtual void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & /*settings*/, + SerializeBinaryBulkStatePtr & /*state*/) const {} + + /// Call before before deserializeBinaryBulkWithMultipleStreams chain to get DeserializeBinaryBulkStatePtr. + virtual void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & /*settings*/, + DeserializeBinaryBulkStatePtr & /*state*/) const {} + /** 'offset' and 'limit' are used to specify range. * limit = 0 - means no limit. * offset must be not greater than size of column. @@ -110,38 +158,24 @@ public: */ virtual void serializeBinaryBulkWithMultipleStreams( const IColumn & column, - OutputStreamGetter getter, size_t offset, size_t limit, - bool /*position_independent_encoding*/, - SubstreamPath path) const + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & /*state*/) const { - if (WriteBuffer * stream = getter(path)) + if (WriteBuffer * stream = settings.getter(settings.path)) serializeBinaryBulk(column, *stream, offset, limit); } - struct DeserializeBinaryBulkState - { - virtual ~DeserializeBinaryBulkState() = default; - }; - using DeserializeBinaryBulkStatePtr = std::shared_ptr; - - virtual DeserializeBinaryBulkStatePtr createDeserializeBinaryBulkState() const { return nullptr; } - - /** Read no more than limit values and append them into column. - * avg_value_size_hint - if not zero, may be used to avoid reallocations while reading column of String type. - */ + /// Read no more than limit values and append them into column. virtual void deserializeBinaryBulkWithMultipleStreams( IColumn & column, - InputStreamGetter getter, size_t limit, - double avg_value_size_hint, - bool /*position_independent_encoding*/, - SubstreamPath path, - const DeserializeBinaryBulkStatePtr & /*state*/) const + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & /*state*/) const { - if (ReadBuffer * stream = getter(path)) - deserializeBinaryBulk(column, *stream, limit, avg_value_size_hint); + if (ReadBuffer * stream = settings.getter(settings.path)) + deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint); } /** Override these methods for data types that require just single stream (most of data types). diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 2ea6879f31c..70246c83ef9 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1601,7 +1601,7 @@ private: auto res_keys = std::move(res.column); auto idx = col_with_dict->getUnique()->uniqueInsertRangeFrom(*res_keys, 0, res_keys->size()); - col_with_dict->getIndexes()->insertRangeFrom(*idx->index(res_indexes, 0), 0, res_indexes->size()); + col_with_dict->getIndexes()->insertRangeFrom(*idx->index(*res_indexes, 0), 0, res_indexes->size()); } else col_with_dict->insertRangeFromFullColumn(*res.column, 0, res.column->size()); @@ -1609,7 +1609,7 @@ private: res.column = std::move(res_column); } else - res.column = res.column->index(res_indexes, 0); + res.column = res.column->index(*res_indexes, 0); }; } diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index aa7031a8340..06afd3d8493 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -329,7 +329,8 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si ErrorCodes::LOGICAL_ERROR); col_with_dict->insertRangeFromFullColumn(*temp_res_col, 0, temp_res_col->size()); - res_col.column = indexes ? col_with_dict->index(indexes, 0) : std::move(col_wit_dict_ptr); + res_col.column = indexes ? col_with_dict->index(*indexes, 0) + : std::move(col_wit_dict_ptr); return; } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index 26f7c2c4493..56b5e804dae 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -546,12 +546,13 @@ void MergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) con for (const NameAndTypePair & name_type : storage.getColumns().getAllPhysical()) { + IDataType::SubstreamPath path; name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) { Poco::File bin_file(getFullPath() + IDataType::getFileNameForStream(name_type.name, substream_path) + ".bin"); if (bin_file.exists()) column_to_size[name_type.name] += bin_file.getSize(); - }, {}); + }, path); } } @@ -597,6 +598,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata) { for (const NameAndTypePair & name_type : columns) { + IDataType::SubstreamPath stream_path; name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) { String file_name = IDataType::getFileNameForStream(name_type.name, substream_path); @@ -608,7 +610,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata) if (!checksums.files.count(bin_file_name)) throw Exception("No " + bin_file_name + " file checksum for column " + name + " in part " + path, ErrorCodes::NO_FILE_IN_DATA_PART); - }, {}); + }, stream_path); } } @@ -678,7 +680,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata) throw Exception("Part " + path + " is broken: marks have different sizes.", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); } - }, {}); + }); } } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 0129254679b..c060f9e7982 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -353,7 +353,8 @@ void MergeTreeReader::addStreams(const String & name, const IDataType & type, co uncompressed_cache, aio_threshold, max_read_buffer_size, profile_callback, clock_type)); }; - type.enumerateStreams(callback, {}); + IDataType::SubstreamPath path; + type.enumerateStreams(callback, path); } @@ -362,36 +363,42 @@ void MergeTreeReader::readData( size_t from_mark, bool continue_reading, size_t max_rows_to_read, bool with_offsets) { - IDataType::InputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer * + auto get_stream_getter = [&](bool stream_for_prefix) -> IDataType::InputStreamGetter { - /// If offsets for arrays have already been read. - if (!with_offsets && path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes) - return nullptr; + return [&](const IDataType::SubstreamPath & path) -> ReadBuffer * + { + /// If offsets for arrays have already been read. + if (!with_offsets && path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes) + return nullptr; - String stream_name = IDataType::getFileNameForStream(name, path); + String stream_name = IDataType::getFileNameForStream(name, path); - auto it = streams.find(stream_name); - if (it == streams.end()) - return nullptr; + auto it = streams.find(stream_name); + if (it == streams.end()) + return nullptr; - Stream & stream = *it->second; + Stream & stream = *it->second; - if (!continue_reading) - stream.seekToMark(from_mark); + if (!continue_reading && !stream_for_prefix) + stream.seekToMark(from_mark); - return stream.data_buffer; + return stream.data_buffer; + }; }; - if (!continue_reading) - deserialize_binary_bulk_state_map[name] = type.createDeserializeBinaryBulkState(); + double & avg_value_size_hint = avg_value_size_hints[name]; + IDataType::DeserializeBinaryBulkSettings settings; + settings.avg_value_size_hint = avg_value_size_hint; if (deserialize_binary_bulk_state_map.count(name) == 0) - throw Exception("DeserializeBinaryBulkState wasn't created for column " + name, ErrorCodes::LOGICAL_ERROR); + { + settings.getter = get_stream_getter(true); + type.deserializeBinaryBulkStatePrefix(settings, deserialize_binary_bulk_state_map[name]); + } - double & avg_value_size_hint = avg_value_size_hints[name]; + settings.getter = get_stream_getter(false); auto & deserialize_state = deserialize_binary_bulk_state_map[name]; - type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, - avg_value_size_hint, true, {}, deserialize_state); + type.deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_state); IDataType::updateAvgValueSizeHint(column, avg_value_size_hint); } diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index e99609b0c09..4a10dc442c9 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -64,7 +64,28 @@ void IMergedBlockOutputStream::addStreams( aio_threshold); }; - type.enumerateStreams(callback, {}); + IDataType::SubstreamPath stream_path; + type.enumerateStreams(callback, stream_path); +} + + +IDataType::OutputStreamGetter IMergedBlockOutputStream::createStreamGetter( + const String & name, OffsetColumns & offset_columns, bool skip_offsets) +{ + return [&, skip_offsets] (const IDataType::SubstreamPath & substream_path) -> WriteBuffer * + { + bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; + if (is_offsets && skip_offsets) + return nullptr; + + String stream_name = IDataType::getFileNameForStream(name, substream_path); + + /// Don't write offsets more than one time for Nested type. + if (is_offsets && offset_columns.count(stream_name)) + return nullptr; + + return &column_streams[stream_name]->compressed; + }; } @@ -73,8 +94,13 @@ void IMergedBlockOutputStream::writeData( const IDataType & type, const IColumn & column, OffsetColumns & offset_columns, - bool skip_offsets) + bool skip_offsets, + IDataType::SerializeBinaryBulkStatePtr & serialization_state) { + IDataType::SerializeBinaryBulkSettings settings; + settings.getter = createStreamGetter(name, offset_columns, skip_offsets); + settings.max_dictionary_size = 1024; + size_t size = column.size(); size_t prev_mark = 0; while (prev_mark < size) @@ -109,25 +135,10 @@ void IMergedBlockOutputStream::writeData( writeIntBinary(stream.plain_hashing.count(), stream.marks); writeIntBinary(stream.compressed.offset(), stream.marks); - }, {}); + }, settings.path); } - IDataType::OutputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & substream_path) -> WriteBuffer * - { - bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; - if (is_offsets && skip_offsets) - return nullptr; - - String stream_name = IDataType::getFileNameForStream(name, substream_path); - - /// Don't write offsets more than one time for Nested type. - if (is_offsets && offset_columns.count(stream_name)) - return nullptr; - - return &column_streams[stream_name]->compressed; - }; - - type.serializeBinaryBulkWithMultipleStreams(column, stream_getter, prev_mark, limit, true, {}); + type.serializeBinaryBulkWithMultipleStreams(column, prev_mark, limit, settings, serialization_state); /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path) @@ -143,7 +154,7 @@ void IMergedBlockOutputStream::writeData( return; column_streams[stream_name]->compressed.nextIfAtEnd(); - }, {}); + }, settings.path); prev_mark += limit; } @@ -157,7 +168,7 @@ void IMergedBlockOutputStream::writeData( String stream_name = IDataType::getFileNameForStream(name, substream_path); offset_columns.insert(stream_name); } - }, {}); + }, settings.path); } @@ -284,6 +295,17 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( const NamesAndTypesList * total_column_list, MergeTreeData::DataPart::Checksums * additional_column_checksums) { + /// Finish columns serialization. + IDataType::SerializeBinaryBulkSettings settings; + settings.max_dictionary_size = 1024; + OffsetColumns offset_columns; + auto it = columns_list.begin(); + for (size_t i = 0; i < columns_list.size(); ++i, ++it) + { + settings.getter = createStreamGetter(it->name, offset_columns, false); + it->type->serializeBinaryBulkStateSuffix(settings, serialization_states[i]); + } + if (!total_column_list) total_column_list = &columns_list; @@ -404,28 +426,44 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm index_columns[i] = primary_columns[i].column->cloneEmpty(); } - /// Now write the data. - for (const auto & it : columns_list) + if (serialization_states.empty()) { - const ColumnWithTypeAndName & column = block.getByName(it.name); + serialization_states.reserve(columns_list.size()); + OffsetColumns tmp_offset_columns; + IDataType::SerializeBinaryBulkSettings settings; + + for (const auto & col : columns_list) + { + settings.getter = createStreamGetter(col.name, tmp_offset_columns, false); + serialization_states.emplace_back(nullptr); + col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back()); + } + } + + /// Now write the data. + auto it = columns_list.begin(); + for (size_t i = 0; i < columns_list.size(); ++i, ++it) + { + const ColumnWithTypeAndName & column = block.getByName(it->name); if (permutation) { - auto primary_column_it = primary_columns_name_to_position.find(it.name); + auto primary_column_it = primary_columns_name_to_position.find(it->name); if (primary_columns_name_to_position.end() != primary_column_it) { - writeData(column.name, *column.type, *primary_columns[primary_column_it->second].column, offset_columns, false); + auto & primary_column = *primary_columns[primary_column_it->second].column; + writeData(column.name, *column.type, primary_column, offset_columns, false, serialization_states[i]); } else { /// We rearrange the columns that are not included in the primary key here; Then the result is released - to save RAM. - ColumnPtr permutted_column = column.column->permute(*permutation, 0); - writeData(column.name, *column.type, *permutted_column, offset_columns, false); + ColumnPtr permuted_column = column.column->permute(*permutation, 0); + writeData(column.name, *column.type, *permuted_column, offset_columns, false, serialization_states[i]); } } else { - writeData(column.name, *column.type, *column.column, offset_columns, false); + writeData(column.name, *column.type, *column.column, offset_columns, false, serialization_states[i]); } } @@ -479,11 +517,21 @@ void MergedColumnOnlyOutputStream::write(const Block & block) if (!initialized) { column_streams.clear(); + serialization_states.clear(); + serialization_states.reserve(block.columns()); + OffsetColumns tmp_offset_columns; + IDataType::SerializeBinaryBulkSettings settings; + for (size_t i = 0; i < block.columns(); ++i) { - addStreams(part_path, block.safeGetByPosition(i).name, - *block.safeGetByPosition(i).type, 0, skip_offsets); + const auto & col = block.safeGetByPosition(i); + + addStreams(part_path, col.name, *col.type, 0, skip_offsets); + serialization_states.emplace_back(nullptr); + settings.getter = createStreamGetter(col.name, tmp_offset_columns, false); + col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back()); } + initialized = true; } @@ -493,7 +541,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) for (size_t i = 0; i < block.columns(); ++i) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); - writeData(column.name, *column.type, *column.column, offset_columns, skip_offsets); + writeData(column.name, *column.type, *column.column, offset_columns, skip_offsets, serialization_states[i]); } size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity; @@ -507,6 +555,17 @@ void MergedColumnOnlyOutputStream::writeSuffix() MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums() { + /// Finish columns serialization. + IDataType::SerializeBinaryBulkSettings settings; + settings.max_dictionary_size = 1024; + OffsetColumns offset_columns; + for (size_t i = 0; i < header.columns(); ++i) + { + auto & column = header.safeGetByPosition(i); + settings.getter = createStreamGetter(column.name, offset_columns, skip_offsets); + column.type->serializeBinaryBulkStateSuffix(settings, serialization_states[i]); + } + MergeTreeData::DataPart::Checksums checksums; for (auto & column_stream : column_streams) @@ -519,6 +578,7 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG } column_streams.clear(); + serialization_states.clear(); initialized = false; return checksums; diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index ea928f59bb5..a7101f844cb 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -25,6 +25,8 @@ public: protected: using OffsetColumns = std::set; + using SerializationState = IDataType::SerializeBinaryBulkStatePtr; + using SerializationStates = std::vector; struct ColumnStream { @@ -64,8 +66,12 @@ protected: void addStreams(const String & path, const String & name, const IDataType & type, size_t estimated_size, bool skip_offsets); + + IDataType::OutputStreamGetter createStreamGetter(const String & name, OffsetColumns & offset_columns, bool skip_offsets); + /// Write data of one column. - void writeData(const String & name, const IDataType & type, const IColumn & column, OffsetColumns & offset_columns, bool skip_offsets); + void writeData(const String & name, const IDataType & type, const IColumn & column, OffsetColumns & offset_columns, + bool skip_offsets, IDataType::SerializeBinaryBulkStatePtr & serialization_state); MergeTreeData & storage; @@ -135,6 +141,7 @@ private: private: NamesAndTypesList columns_list; + SerializationStates serialization_states; String part_path; size_t rows_count = 0; @@ -161,6 +168,7 @@ public: private: Block header; + SerializationStates serialization_states; String part_path; bool initialized = false; diff --git a/dbms/src/Storages/MergeTree/checkDataPart.cpp b/dbms/src/Storages/MergeTree/checkDataPart.cpp index 9cf89bb1077..a978bdf34f7 100644 --- a/dbms/src/Storages/MergeTree/checkDataPart.cpp +++ b/dbms/src/Storages/MergeTree/checkDataPart.cpp @@ -249,6 +249,8 @@ MergeTreeData::DataPart::Checksums checkDataPart( while (true) { + IDataType::DeserializeBinaryBulkSettings settings; + /// Check that mark points to current position in file. bool marks_eof = false; name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) @@ -270,7 +272,7 @@ MergeTreeData::DataPart::Checksums checkDataPart( + ", mrk file offset: " + toString(stream.mrk_hashing_buf.count())); throw; } - }, {}); + }, settings.path); ++mark_num; @@ -278,19 +280,18 @@ MergeTreeData::DataPart::Checksums checkDataPart( /// NOTE Shared array sizes of Nested columns are read more than once. That's Ok. MutableColumnPtr tmp_column = name_type.type->createColumn(); - auto state = name_type.type->createDeserializeBinaryBulkState(); - name_type.type->deserializeBinaryBulkWithMultipleStreams( - *tmp_column, - [&](const IDataType::SubstreamPath & substream_path) - { - String file_name = IDataType::getFileNameForStream(name_type.name, substream_path); - auto stream_it = streams.find(file_name); - if (stream_it == streams.end()) - throw Exception("Logical error: cannot find stream " + file_name); - return &stream_it->second.uncompressed_hashing_buf; - }, - index_granularity, - 0, true, {}, state); + settings.getter = [&](const IDataType::SubstreamPath & substream_path) + { + String file_name = IDataType::getFileNameForStream(name_type.name, substream_path); + auto stream_it = streams.find(file_name); + if (stream_it == streams.end()) + throw Exception("Logical error: cannot find stream " + file_name); + return &stream_it->second.uncompressed_hashing_buf; + }; + + IDataType::DeserializeBinaryBulkStatePtr state; + name_type.type->deserializeBinaryBulkStatePrefix(settings, state); + name_type.type->deserializeBinaryBulkWithMultipleStreams(*tmp_column, index_granularity, settings, state); size_t read_size = tmp_column->size(); column_size += read_size; diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 54c1323fcc7..e35d3ca17c2 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -76,6 +76,7 @@ protected: Block readImpl() override; private: + size_t block_size; NamesAndTypesList columns; StorageLog & storage; @@ -101,6 +102,10 @@ private: using FileStreams = std::map; FileStreams streams; + using DeserializeState = IDataType::DeserializeBinaryBulkStatePtr; + using DeserializeStates = std::map; + DeserializeStates deserialize_states; + void readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read); }; @@ -167,6 +172,12 @@ private: WriteBufferFromFile marks_stream; /// Declared below `lock` to make the file open when rwlock is captured. + using SerializeState = IDataType::SerializeBinaryBulkStatePtr; + using SerializeStates = std::map; + SerializeStates serialize_states; + + IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenStreams & written_streams); + void writeData(const String & name, const IDataType & type, const IColumn & column, MarksForColumns & out_marks, WrittenStreams & written_streams); @@ -225,26 +236,36 @@ Block LogBlockInputStream::readImpl() void LogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read) { - IDataType::InputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer * + IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. + + auto createStringGetter = [&](bool stream_for_prefix) { - String stream_name = IDataType::getFileNameForStream(name, path); + return [&] (const IDataType::SubstreamPath & path) -> ReadBuffer * + { + String stream_name = IDataType::getFileNameForStream(name, path); - const auto & file_it = storage.files.find(stream_name); - if (storage.files.end() == file_it) - throw Exception("Logical error: no information about file " + stream_name + " in StorageLog", ErrorCodes::LOGICAL_ERROR); + const auto & file_it = storage.files.find(stream_name); + if (storage.files.end() == file_it) + throw Exception("Logical error: no information about file " + stream_name + " in StorageLog", ErrorCodes::LOGICAL_ERROR); - auto it = streams.try_emplace(stream_name, - file_it->second.data_file.path(), - mark_number - ? file_it->second.marks[mark_number].offset - : 0, - max_read_buffer_size).first; + UInt64 offset = 0; + if (!stream_for_prefix && mark_number) + offset = file_it->second.marks[mark_number].offset; - return &it->second.compressed; + auto & data_file_path = file_it->second.data_file.path(); + auto it = streams.try_emplace(stream_name, data_file_path, offset, max_read_buffer_size).first; + return &it->second.compressed; + }; }; - auto state = type.createDeserializeBinaryBulkState(); - type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, 0, true, {}, state); /// TODO Use avg_value_size_hint. + if (deserialize_states.count(name) == 0) + { + settings.getter = createStringGetter(true); + type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); + } + + settings.getter = createStringGetter(false); + type.deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_states[name]); } @@ -274,6 +295,18 @@ void LogBlockOutputStream::writeSuffix() return; done = true; + WrittenStreams written_streams; + IDataType::SerializeBinaryBulkSettings settings; + for (const auto & column : getHeader()) + { + auto it = serialize_states.find(column.name); + if (it != serialize_states.end()) + { + settings.getter = createStreamGetter(column.name, written_streams); + column.type->serializeBinaryBulkStateSuffix(settings, it->second); + } + } + /// Finish write. marks_stream.next(); @@ -291,27 +324,10 @@ void LogBlockOutputStream::writeSuffix() } -void LogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, - MarksForColumns & out_marks, - WrittenStreams & written_streams) +IDataType::OutputStreamGetter LogBlockOutputStream::createStreamGetter(const String & name, + WrittenStreams & written_streams) { - type.enumerateStreams([&] (const IDataType::SubstreamPath & path) - { - String stream_name = IDataType::getFileNameForStream(name, path); - if (written_streams.count(stream_name)) - return; - - const auto & file = storage.files[stream_name]; - const auto stream_it = streams.try_emplace(stream_name, storage.files[stream_name].data_file.path(), storage.max_compress_block_size).first; - - Mark mark; - mark.rows = (file.marks.empty() ? 0 : file.marks.back().rows) + column.size(); - mark.offset = stream_it->second.plain_offset + stream_it->second.plain.count(); - - out_marks.emplace_back(file.column_index, mark); - }, {}); - - IDataType::OutputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> WriteBuffer * + return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer * { String stream_name = IDataType::getFileNameForStream(name, path); if (written_streams.count(stream_name)) @@ -319,11 +335,50 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type auto it = streams.find(stream_name); if (streams.end() == it) - throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", ErrorCodes::LOGICAL_ERROR); + throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", + ErrorCodes::LOGICAL_ERROR); return &it->second.compressed; }; +} - type.serializeBinaryBulkWithMultipleStreams(column, stream_getter, 0, 0, true, {}); + +void LogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, + MarksForColumns & out_marks, + WrittenStreams & written_streams) +{ + IDataType::SerializeBinaryBulkSettings settings; + + type.enumerateStreams([&] (const IDataType::SubstreamPath & path) + { + String stream_name = IDataType::getFileNameForStream(name, path); + if (written_streams.count(stream_name)) + return; + + streams.try_emplace(stream_name, storage.files[stream_name].data_file.path(), storage.max_compress_block_size); + }, settings.path); + + settings.getter = createStreamGetter(name, written_streams); + + if (serialize_states.count(name) == 0) + type.serializeBinaryBulkStatePrefix(settings, serialize_states[name]); + + type.enumerateStreams([&] (const IDataType::SubstreamPath & path) + { + String stream_name = IDataType::getFileNameForStream(name, path); + if (written_streams.count(stream_name)) + return; + + const auto & file = storage.files[stream_name]; + const auto stream_it = streams.find(stream_name); + + Mark mark; + mark.rows = (file.marks.empty() ? 0 : file.marks.back().rows) + column.size(); + mark.offset = stream_it->second.plain_offset + stream_it->second.plain.count(); + + out_marks.emplace_back(file.column_index, mark); + }, settings.path); + + type.serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); type.enumerateStreams([&] (const IDataType::SubstreamPath & path) { @@ -335,7 +390,7 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type if (streams.end() == it) throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", ErrorCodes::LOGICAL_ERROR); it->second.compressed.next(); - }, {}); + }, settings.path); } @@ -401,7 +456,8 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type) } }; - type.enumerateStreams(stream_callback, {}); + IDataType::SubstreamPath path; + type.enumerateStreams(stream_callback, path); } @@ -474,11 +530,12 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount() const * If this is a data type with multiple stream, get the first stream, that we assume have real row count. * (Example: for Array data type, first stream is array sizes; and number of array sizes is the number of arrays). */ + IDataType::SubstreamPath path; column_type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path) { if (filename.empty()) filename = IDataType::getFileNameForStream(column_name, substream_path); - }, {}); + }, path); Files_t::const_iterator it = files.find(filename); if (files.end() == it) diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index ab4758a2981..7515c85b87f 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -95,6 +95,10 @@ private: using FileStreams = std::map>; FileStreams streams; + using DeserializeState = IDataType::DeserializeBinaryBulkStatePtr; + using DeserializeStates = std::map; + DeserializeStates deserialize_states; + void readData(const String & name, const IDataType & type, IColumn & column, size_t limit); }; @@ -149,8 +153,13 @@ private: using FileStreams = std::map>; FileStreams streams; + using SerializeState = IDataType::SerializeBinaryBulkStatePtr; + using SerializeStates = std::map; + SerializeStates serialize_states; + using WrittenStreams = std::set; + IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenStreams & written_streams); void writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams); }; @@ -206,7 +215,8 @@ Block TinyLogBlockInputStream::readImpl() void TinyLogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t limit) { - IDataType::InputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer * + IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. + settings.getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer * { String stream_name = IDataType::getFileNameForStream(name, path); @@ -216,14 +226,17 @@ void TinyLogBlockInputStream::readData(const String & name, const IDataType & ty return &streams[stream_name]->compressed; }; - auto state = type.createDeserializeBinaryBulkState(); - type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, limit, 0, true, {}, state); /// TODO Use avg_value_size_hint. + if (deserialize_states.count(name) == 0) + type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); + + type.deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name]); } -void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams) +IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter(const String & name, + WrittenStreams & written_streams) { - IDataType::OutputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> WriteBuffer * + return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer * { String stream_name = IDataType::getFileNameForStream(name, path); @@ -231,12 +244,23 @@ void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & return nullptr; if (!streams.count(stream_name)) - streams[stream_name] = std::make_unique(storage.files[stream_name].data_file.path(), storage.max_compress_block_size); + streams[stream_name] = std::make_unique(storage.files[stream_name].data_file.path(), + storage.max_compress_block_size); return &streams[stream_name]->compressed; }; +} - type.serializeBinaryBulkWithMultipleStreams(column, stream_getter, 0, 0, true, {}); + +void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams) +{ + IDataType::SerializeBinaryBulkSettings settings; + settings.getter = createStreamGetter(name, written_streams); + + if (serialize_states.count(name) == 0) + type.serializeBinaryBulkStatePrefix(settings, serialize_states[name]); + + type.serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); } @@ -246,6 +270,18 @@ void TinyLogBlockOutputStream::writeSuffix() return; done = true; + WrittenStreams written_streams; + IDataType::SerializeBinaryBulkSettings settings; + for (const auto & column : getHeader()) + { + auto it = serialize_states.find(column.name); + if (it != serialize_states.end()) + { + settings.getter = createStreamGetter(column.name, written_streams); + column.type->serializeBinaryBulkStateSuffix(settings, it->second); + } + } + /// Finish write. for (auto & stream : streams) stream.second->finalize(); @@ -321,7 +357,8 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type } }; - type.enumerateStreams(stream_callback, {}); + IDataType::SubstreamPath path; + type.enumerateStreams(stream_callback, path); } From 9e9b33ebe5b9d2b38932e0ad71513e7bf7851897 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Jun 2018 16:37:35 +0300 Subject: [PATCH 097/315] Fix deserialization indexes for additional keys in DataTypeWithDictionary. --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 0ec0c355e9f..9a692cb508d 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -354,12 +354,11 @@ namespace MutableColumnPtr mapIndexWithOverflow(PaddedPODArray & index, size_t max_val) { HashMap hash_map; - HashMap hash_map_with_overflow; for (auto val : index) { - auto & map = val < max_val ? hash_map : hash_map_with_overflow; - map.insert({val, map.size()}); + if (val < max_val) + hash_map.insert({val, hash_map.size()}); } auto index_map_col = ColumnVector::create(); @@ -371,7 +370,7 @@ namespace for (auto & val : index) val = val < max_val ? hash_map[val] - : hash_map_with_overflow[val] + hash_map.size(); + : val - max_val + hash_map.size(); return index_map_col; } From f191eb8ae9cf9461363be63c2d7b725fd24ea636 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Jun 2018 18:18:17 +0300 Subject: [PATCH 098/315] Fix ColumnWithDictionary::getPremutation limit --- dbms/src/Columns/ColumnWithDictionary.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index 7d665cc9a1c..cfa847809ba 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -165,6 +165,9 @@ public: void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override { + if (limit == 0) + limit = size(); + size_t unique_limit = std::min(limit, getUnique()->size()); Permutation unique_perm; getUnique()->getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); From b1eace28773403cd60ac03d7c6466574f3b80c3a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jun 2018 16:10:46 +0300 Subject: [PATCH 099/315] Fixed DataTypeWithDictionary deserialization. --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 9a692cb508d..17d8b96e892 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -549,6 +549,17 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( } else { + if (column_with_global_dictionary) + { + auto unique_indexes = mapUniqueIndex(*column_with_dictionary.getIndexes()); + auto sub_keys = column_with_dictionary.getUnique()->getNestedColumn()->index(*unique_indexes, 0); + auto new_unique = createColumnUnique(*dictionary_type, *indexes_type); + auto new_idx = new_unique->uniqueInsertRangeFrom(*sub_keys, 0, sub_keys->size()); + column_with_dictionary.setUnique(std::move(new_unique)); + column_with_dictionary.setIndexes((*(new_idx->index(*column_with_dictionary.getIndexes(), 0))).mutate()); + column_unique = column_with_dictionary.getUnique(); + } + auto index_map = mapIndexWithOverflow(*indexes_column, global_dictionary->size()); auto used_keys = global_dictionary->getNestedColumn()->index(*index_map, 0); auto indexes = column_unique->uniqueInsertRangeFrom(*used_keys, 0, used_keys->size()); From bf3327da8be5564284a635d0f15008e6fb98ee13 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jun 2018 20:07:08 +0300 Subject: [PATCH 100/315] Removed nullable column from ColumnUnique. Added IColumnUnique::getNestedNotNullableColumn(). --- dbms/src/Columns/ColumnUnique.h | 75 ++++++++++++------------- dbms/src/Columns/ColumnWithDictionary.h | 3 +- dbms/src/Columns/IColumnUnique.h | 4 +- 3 files changed, 40 insertions(+), 42 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 06c7f10a184..2fba042b021 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -54,14 +54,14 @@ class ColumnUnique final : public COWPtrHelper>; private: - explicit ColumnUnique(MutableColumnPtr && holder); + explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable); explicit ColumnUnique(const IDataType & type); - ColumnUnique(const ColumnUnique & other) - : column_holder(other.column_holder), nullable_column(other.nullable_column) - , nullable_column_map(other.nullable_column_map), is_nullable(other.is_nullable) {} + ColumnUnique(const ColumnUnique & other) : column_holder(other.column_holder), is_nullable(other.is_nullable) {} public: - const ColumnPtr & getNestedColumn() const override; + ColumnPtr getNestedColumn() const override; + const ColumnPtr & getNestedNotNullableColumn() const override { return column_holder; } + size_t uniqueInsert(const Field & x) override; size_t uniqueInsertFrom(const IColumn & src, size_t n) override; MutableColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; @@ -113,18 +113,11 @@ public: { return column_holder->allocatedBytes() + (index ? index->getBufferSizeInBytes() : 0) - + (nullable_column ? nullable_column->allocatedBytes() : 0); + + (cached_null_mask ? cached_null_mask->allocatedBytes() : 0); } void forEachSubcolumn(IColumn::ColumnCallback callback) override { - callback(is_nullable ? nullable_column : column_holder); - /// If column was mutated, we need to restore ptrs. - if (is_nullable) - { - auto & column_nullable = static_cast(nullable_column->assumeMutableRef()); - column_holder = column_nullable.getNestedColumnPtr(); - nullable_column_map = &column_nullable.getNullMapData(); - } + callback(column_holder); } private: @@ -133,9 +126,8 @@ private: ColumnPtr column_holder; - /// For DataTypeNullable, nullptr otherwise. - ColumnPtr nullable_column; - NullMap * nullable_column_map = nullptr; + /// For DataTypeNullable, stores null map. + mutable ColumnPtr cached_null_mask; /// Lazy initialized. std::unique_ptr index; @@ -146,7 +138,7 @@ private: void buildIndex(); ColumnType * getRawColumnPtr() { return static_cast(column_holder->assumeMutable().get()); } - const ColumnType * getRawColumnPtr() const { return static_cast(column_holder.get()); } + const ColumnType * getRawColumnPtr() const { return static_cast(column_holder.get()); } IndexType insertIntoMap(const StringRefWrapper & ref, IndexType value); void uniqueInsertRangeImpl( @@ -161,38 +153,41 @@ private: template ColumnUnique::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable()) { - if (is_nullable) - { - nullable_column = type.createColumn()->cloneResized(numSpecialValues()); - auto & column_nullable = static_cast(nullable_column->assumeMutableRef()); - column_holder = column_nullable.getNestedColumnPtr(); - nullable_column_map = &column_nullable.getNullMapData(); - (*nullable_column_map)[getDefaultValueIndex()] = 0; - } - else - column_holder = type.createColumn()->cloneResized(numSpecialValues()); + const auto & holder_type = is_nullable ? *static_cast(type).getNestedType() : type; + column_holder = holder_type.createColumn()->cloneResized(numSpecialValues()); } template -ColumnUnique::ColumnUnique(MutableColumnPtr && holder) : column_holder(std::move(holder)) +ColumnUnique::ColumnUnique(MutableColumnPtr && holder, bool is_nullable) + : column_holder(std::move(holder)), is_nullable(is_nullable) { + if (column_holder->size() < numSpecialValues()) + throw Exception("Too small holder column for ColumnUnique.", ErrorCodes::ILLEGAL_COLUMN); if (column_holder->isColumnNullable()) - { - nullable_column = std::move(column_holder); - auto & column_nullable = static_cast(nullable_column->assumeMutableRef()); - column_holder = column_nullable.getNestedColumnPtr(); - nullable_column_map = &column_nullable.getNullMapData(); - is_nullable = true; - } + throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN); } template -const ColumnPtr& ColumnUnique::getNestedColumn() const +ColumnPtr ColumnUnique::getNestedColumn() const { if (is_nullable) { - nullable_column_map->resize_fill(column_holder->size()); - return nullable_column; + size_t size = getRawColumnPtr()->size(); + if (!cached_null_mask) + { + ColumnUInt8::MutablePtr null_mask = ColumnUInt8::create(size, UInt8(0)); + null_mask->getData()[getNullValueIndex()] = 1; + cached_null_mask = std::move(null_mask); + } + + if (cached_null_mask->size() != size) + { + MutableColumnPtr null_mask = (*std::move(cached_null_mask)).mutate(); + static_cast(*null_mask).getData().resize_fill(size); + cached_null_mask = std::move(null_mask); + } + + return ColumnNullable::create(column_holder, cached_null_mask); } return column_holder; } @@ -201,7 +196,7 @@ template size_t ColumnUnique::getNullValueIndex() const { if (!is_nullable) - throw Exception("ColumnUnique can't contain null values."); + throw Exception("ColumnUnique can't contain null values.", ErrorCodes::LOGICAL_ERROR); return 0; } diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index cfa847809ba..f363dbe793f 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -100,7 +100,8 @@ public: throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); auto & src_with_dict = static_cast(src); - auto & src_nested = src_with_dict.getUnique()->getNestedColumn(); + /// TODO: Support native insertion from other unique column. It will help to avoid null map creation. + auto src_nested = src_with_dict.getUnique()->getNestedColumn(); auto inserted_idx = getUnique()->uniqueInsertRangeFrom(*src_nested, 0, src_nested->size()); auto idx = inserted_idx->index(*src_with_dict.getIndexes()->cut(start, length), 0); getIndexes()->insertRangeFrom(*idx, 0, length); diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index 15fe1ef6c81..aedaf3fed1f 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -12,7 +12,9 @@ public: /// Column always contains Null if it's Nullable and empty string if it's String or Nullable(String). /// So, size may be greater than the number of inserted unique values. - virtual const ColumnPtr & getNestedColumn() const = 0; + virtual ColumnPtr getNestedColumn() const = 0; + /// The same as getNestedColumn, but removes null map if nested column is nullable. + virtual const ColumnPtr & getNestedNotNullableColumn() const = 0; size_t size() const override { return getNestedColumn()->size(); } From 1381369638877fe1d2c01434b3b17dbc962bf285 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 28 Jun 2018 20:48:05 +0300 Subject: [PATCH 101/315] Update Defines.h --- dbms/src/Core/Defines.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 877ad09f839..f2576ff8b30 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -61,7 +61,7 @@ #define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226 #define DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE 54337 #define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372 -#define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54386 +#define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54389 /// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change. #define DBMS_TCP_PROTOCOL_VERSION 54226 From 3894f094ea53a9411e30ed62ebcd1f7dec244086 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 28 Jun 2018 22:40:18 +0300 Subject: [PATCH 102/315] Fixed setting descriptions #2482 --- dbms/src/Interpreters/Settings.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index beafc3e09c8..84b3dc33e78 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -264,9 +264,9 @@ struct Settings M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.") \ M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \ M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \ - M(SettingBool, log_profile_events, true, "Log query settings into the query_log and query_thread_log.") \ - M(SettingBool, log_query_settings, true, "Log query performance statistics into the query_log.") \ - M(SettingBool, log_query_threads, true, "Log query threads into system.query_log table.") \ + M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \ + M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \ + M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table.") \ M(SettingString, server_logs_level, "none", "Send internal server logs to client.") From d8654d07e1a73cbeb8de306544ba4904e0512e76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 28 Jun 2018 22:43:27 +0300 Subject: [PATCH 103/315] Auto version update to [54389] --- dbms/cmake/version.cmake | 6 +++--- debian/changelog | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 29abf098b26..adc80346d8a 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -1,7 +1,7 @@ # This strings autochanged from release_lib.sh: -set(VERSION_DESCRIBE v1.1.54388-testing) -set(VERSION_REVISION 54388) -set(VERSION_GITHASH 2447755700f40af317cb80ba8800b94d6350d148) +set(VERSION_DESCRIBE v1.1.54389-testing) +set(VERSION_REVISION 54389) +set(VERSION_GITHASH 3894f094ea53a9411e30ed62ebcd1f7dec244086) # end of autochange set (VERSION_MAJOR 1) diff --git a/debian/changelog b/debian/changelog index dc05db4aa45..5c1ce4e2e42 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (1.1.54388) unstable; urgency=low +clickhouse (1.1.54389) unstable; urgency=low * Modified source code - -- Wed, 27 Jun 2018 16:10:59 +0300 + -- Alexey Milovidov Thu, 28 Jun 2018 22:43:27 +0300 From 5d00a23f473b47b490b379b0776f30134866d6c5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 28 Jun 2018 23:04:13 +0300 Subject: [PATCH 104/315] Leave thread number in log in a consistent place #2482 --- libs/libdaemon/include/daemon/ExtendedLogChannel.h | 1 - libs/libdaemon/include/daemon/OwnFormattingChannel.h | 4 +--- libs/libdaemon/src/OwnPatternFormatter.cpp | 10 ++++++---- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/libs/libdaemon/include/daemon/ExtendedLogChannel.h b/libs/libdaemon/include/daemon/ExtendedLogChannel.h index 6d03970676a..b90494273f2 100644 --- a/libs/libdaemon/include/daemon/ExtendedLogChannel.h +++ b/libs/libdaemon/include/daemon/ExtendedLogChannel.h @@ -11,7 +11,6 @@ namespace DB class ExtendedLogMessage { public: - explicit ExtendedLogMessage(const Poco::Message & base) : base(base) {} /// Attach additional data to the message diff --git a/libs/libdaemon/include/daemon/OwnFormattingChannel.h b/libs/libdaemon/include/daemon/OwnFormattingChannel.h index bcce42df866..e206debb8e5 100644 --- a/libs/libdaemon/include/daemon/OwnFormattingChannel.h +++ b/libs/libdaemon/include/daemon/OwnFormattingChannel.h @@ -16,9 +16,8 @@ namespace DB class OwnFormattingChannel : public Poco::Channel, public ExtendedLogChannel { public: - explicit OwnFormattingChannel(Poco::AutoPtr pFormatter_ = nullptr, Poco::AutoPtr pChannel_ = nullptr) - : pFormatter(std::move(pFormatter_)), pChannel(std::move(pChannel_)) {} + : pFormatter(std::move(pFormatter_)), pChannel(std::move(pChannel_)) {} void setChannel(Poco::AutoPtr pChannel_) { @@ -48,7 +47,6 @@ public: ~OwnFormattingChannel() override; private: - Poco::AutoPtr pFormatter; Poco::AutoPtr pChannel; Poco::Message::Priority priority = Poco::Message::PRIO_TRACE; diff --git a/libs/libdaemon/src/OwnPatternFormatter.cpp b/libs/libdaemon/src/OwnPatternFormatter.cpp index 310cc1d9fb9..55e7eb7789a 100644 --- a/libs/libdaemon/src/OwnPatternFormatter.cpp +++ b/libs/libdaemon/src/OwnPatternFormatter.cpp @@ -45,16 +45,18 @@ void OwnPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ext, DB::writeChar('0' + ((msg_ext.time_microseconds / 10) % 10), wb); DB::writeChar('0' + ((msg_ext.time_microseconds / 1) % 10), wb); + writeCString(" [ ", wb); + DB::writeIntText(msg_ext.thread_number, wb); + writeCString(" ] ", wb); + if (!msg_ext.query_id.empty()) { - writeCString(" {", wb); + writeCString("{", wb); DB::writeString(msg_ext.query_id, wb); writeCString("}", wb); } - writeCString(" [ ", wb); - DB::writeIntText(msg_ext.thread_number, wb); - writeCString(" ] <", wb); + writeCString(" <", wb); DB::writeString(getPriorityName(static_cast(msg.getPriority())), wb); writeCString("> ", wb); DB::writeString(msg.getSource(), wb); From e051998e1cfa53ae750d55d1126f843e61de2d24 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 28 Jun 2018 23:11:14 +0300 Subject: [PATCH 105/315] Renamed setting "send_logs_level" #2482 --- dbms/programs/server/TCPHandler.cpp | 4 ++-- dbms/src/Interpreters/Settings.h | 2 +- dbms/tests/queries/shell_config.sh | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 40971486c8a..b6961c12a4d 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -156,10 +156,10 @@ void TCPHandler::runImpl() /// Should we send internal logs to client? if (client_revision >= DBMS_MIN_REVISION_WITH_SERVER_LOGS - && query_context.getSettingsRef().server_logs_level.value != "none") + && query_context.getSettingsRef().send_logs_level.value != "none") { state.logs_queue = std::make_shared(); - state.logs_queue->max_priority = Poco::Logger::parseLevel(query_context.getSettingsRef().server_logs_level.value); + state.logs_queue->max_priority = Poco::Logger::parseLevel(query_context.getSettingsRef().send_logs_level.value); CurrentThread::attachInternalTextLogsQueue(state.logs_queue); } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 84b3dc33e78..5a1e2ba9768 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -267,7 +267,7 @@ struct Settings M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table.") \ - M(SettingString, server_logs_level, "none", "Send internal server logs to client.") + M(SettingString, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'info', 'warning', 'error', 'none'") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ diff --git a/dbms/tests/queries/shell_config.sh b/dbms/tests/queries/shell_config.sh index ea5e55895f6..0078c3432ad 100644 --- a/dbms/tests/queries/shell_config.sh +++ b/dbms/tests/queries/shell_config.sh @@ -2,7 +2,7 @@ export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"} export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="${CLICKHOUSE_BINARY}-client"} export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL:="warning"} -export CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --server_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}" +export CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}" export CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"} export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:="/etc/clickhouse-server/config.xml"} From 158742831ee2fbfb47ee846cb4fe89ef85488210 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Jul 2018 16:29:31 +0300 Subject: [PATCH 106/315] Little better #2482 --- libs/libdaemon/src/OwnPatternFormatter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/libs/libdaemon/src/OwnPatternFormatter.cpp b/libs/libdaemon/src/OwnPatternFormatter.cpp index 55e7eb7789a..3bc706634d9 100644 --- a/libs/libdaemon/src/OwnPatternFormatter.cpp +++ b/libs/libdaemon/src/OwnPatternFormatter.cpp @@ -53,10 +53,10 @@ void OwnPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ext, { writeCString("{", wb); DB::writeString(msg_ext.query_id, wb); - writeCString("}", wb); + writeCString("} ", wb); } - writeCString(" <", wb); + writeCString("<", wb); DB::writeString(getPriorityName(static_cast(msg.getPriority())), wb); writeCString("> ", wb); DB::writeString(msg.getSource(), wb); From 4b099e385b4694fc88ca554f1223da9ce4d6d60e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Jul 2018 20:28:06 +0300 Subject: [PATCH 107/315] Fixed typo #2482 --- dbms/src/Common/ThreadStatus.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 3538201dda3..276f783519b 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -97,7 +97,7 @@ public: String getQueryID(); - /// Starts new query and create new thread group fro it, current thread becomes master thread of the query + /// Starts new query and create new thread group for it, current thread becomes master thread of the query void initializeQuery(); /// Attaches slave thread to existing thread group From c542cb6314748f0fef9f058ef1f3d934a0b6716d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 Jun 2018 21:12:29 +0300 Subject: [PATCH 108/315] Expanding indexes for ColumnUnique. --- dbms/src/Columns/ColumnUnique.h | 189 ++++++++++++++++------ dbms/src/Columns/ColumnWithDictionary.cpp | 88 ++++++++++ dbms/src/Columns/ColumnWithDictionary.h | 23 ++- 3 files changed, 233 insertions(+), 67 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 2fba042b021..e1988723640 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -7,6 +7,7 @@ #include #include #include +#include class NullMap; @@ -48,10 +49,10 @@ namespace ZeroTraits namespace DB { -template -class ColumnUnique final : public COWPtrHelper> +template +class ColumnUnique final : public COWPtrHelper> { - friend class COWPtrHelper>; + friend class COWPtrHelper; private: explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable); @@ -122,7 +123,7 @@ public: private: - using IndexMapType = HashMap, IndexType, StringRefHash>; + using IndexMapType = HashMap, UInt64, StringRefHash>; ColumnPtr column_holder; @@ -139,26 +140,27 @@ private: void buildIndex(); ColumnType * getRawColumnPtr() { return static_cast(column_holder->assumeMutable().get()); } const ColumnType * getRawColumnPtr() const { return static_cast(column_holder.get()); } - IndexType insertIntoMap(const StringRefWrapper & ref, IndexType value); + UInt64 insertIntoMap(const StringRefWrapper & ref, UInt64 value); - void uniqueInsertRangeImpl( + template + MutableColumnPtr uniqueInsertRangeImpl( const IColumn & src, size_t start, size_t length, - typename ColumnVector::Container & positions, + typename ColumnVector::MutablePtr && positions_column, ColumnType * overflowed_keys, size_t max_dictionary_size); }; -template -ColumnUnique::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable()) +template +ColumnUnique::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable()) { const auto & holder_type = is_nullable ? *static_cast(type).getNestedType() : type; column_holder = holder_type.createColumn()->cloneResized(numSpecialValues()); } -template -ColumnUnique::ColumnUnique(MutableColumnPtr && holder, bool is_nullable) +template +ColumnUnique::ColumnUnique(MutableColumnPtr && holder, bool is_nullable) : column_holder(std::move(holder)), is_nullable(is_nullable) { if (column_holder->size() < numSpecialValues()) @@ -167,8 +169,8 @@ ColumnUnique::ColumnUnique(MutableColumnPtr && holder, bo throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN); } -template -ColumnPtr ColumnUnique::getNestedColumn() const +template +ColumnPtr ColumnUnique::getNestedColumn() const { if (is_nullable) { @@ -192,8 +194,8 @@ ColumnPtr ColumnUnique::getNestedColumn() const return column_holder; } -template -size_t ColumnUnique::getNullValueIndex() const +template +size_t ColumnUnique::getNullValueIndex() const { if (!is_nullable) throw Exception("ColumnUnique can't contain null values.", ErrorCodes::LOGICAL_ERROR); @@ -201,8 +203,8 @@ size_t ColumnUnique::getNullValueIndex() const return 0; } -template -void ColumnUnique::buildIndex() +template +void ColumnUnique::buildIndex() { if (index) return; @@ -216,8 +218,8 @@ void ColumnUnique::buildIndex() } } -template -IndexType ColumnUnique::insertIntoMap(const StringRefWrapper & ref, IndexType value) +template +IndexType ColumnUnique::insertIntoMap(const StringRefWrapper & ref, IndexType value) { if (!index) buildIndex(); @@ -233,8 +235,8 @@ IndexType ColumnUnique::insertIntoMap(const StringRefWrap return it->second; } -template -size_t ColumnUnique::uniqueInsert(const Field & x) +template +size_t ColumnUnique::uniqueInsert(const Field & x) { if (x.getType() == Field::Types::Null) return getNullValueIndex(); @@ -253,8 +255,8 @@ size_t ColumnUnique::uniqueInsert(const Field & x) return pos; } -template -size_t ColumnUnique::uniqueInsertFrom(const IColumn & src, size_t n) +template +size_t ColumnUnique::uniqueInsertFrom(const IColumn & src, size_t n) { if (is_nullable && src.isNullAt(n)) return getNullValueIndex(); @@ -263,8 +265,8 @@ size_t ColumnUnique::uniqueInsertFrom(const IColumn & src return uniqueInsertData(ref.data, ref.size); } -template -size_t ColumnUnique::uniqueInsertData(const char * pos, size_t length) +template +size_t ColumnUnique::uniqueInsertData(const char * pos, size_t length) { if (!index) buildIndex(); @@ -274,7 +276,7 @@ size_t ColumnUnique::uniqueInsertData(const char * pos, s if (column->getDataAt(getDefaultValueIndex()) == StringRef(pos, length)) return getDefaultValueIndex(); - auto size = static_cast(column->size()); + UInt64 size = column->size(); auto iter = index->find(StringRefWrapper(StringRef(pos, length))); if (iter == index->end()) @@ -286,8 +288,8 @@ size_t ColumnUnique::uniqueInsertData(const char * pos, s return iter->second; } -template -size_t ColumnUnique::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) +template +size_t ColumnUnique::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) { if (std::is_same::value) return uniqueInsertData(pos, length - 1); @@ -314,8 +316,8 @@ size_t ColumnUnique::uniqueInsertDataWithTerminatingZero( return static_cast(position); } -template -size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) +template +size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) { auto column = getRawColumnPtr(); size_t prev_size = column->size(); @@ -334,12 +336,13 @@ size_t ColumnUnique::uniqueDeserializeAndInsertFromArena( return static_cast(index_pos); } -template -void ColumnUnique::uniqueInsertRangeImpl( +template +template +MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( const IColumn & src, size_t start, size_t length, - typename ColumnVector::Container & positions, + typename ColumnVector::MutablePtr && positions_column, ColumnType * overflowed_keys, size_t max_dictionary_size) { @@ -348,6 +351,35 @@ void ColumnUnique::uniqueInsertRangeImpl( const ColumnType * src_column; const NullMap * null_map = nullptr; + auto & positions = positions_column->getData(); + + using SuperiorIndexType = NumberTraits::Construct::Type; + auto updatePosition = [&](UInt64 & next_position, UInt64 num_added_rows) -> MutableColumnPtr + { + ++next_position; + + if (next_position > std::numeric_limits::max()) + { + if (sizeof(SuperiorIndexType) == sizeof(IndexType)) + throw Exception("Can't find superior index type for type " + demangle(typeid(IndexType).name()), + ErrorCodes::LOGICAL_ERROR); + + auto expanded_column = ColumnVector::create(length); + auto & expanded_data = expanded_column->getData(); + for (size_t i = 0; i < num_added_rows; ++i) + expanded_data[i] = positions[i]; + + return uniqueInsertRangeImpl( + src, + start + num_added_rows, + length - num_added_rows, + std::move(expanded_column), + overflowed_keys, + max_dictionary_size); + } + + return nullptr; + }; if (src.isColumnNullable()) { @@ -364,7 +396,7 @@ void ColumnUnique::uniqueInsertRangeImpl( auto column = getRawColumnPtr(); - size_t next_position = column->size(); + UInt64 next_position = column->size(); for (auto i : ext::range(0, length)) { auto row = start + i; @@ -388,7 +420,9 @@ void ColumnUnique::uniqueInsertRangeImpl( auto ref = src_column->getDataAt(row); overflowed_keys->insertData(ref.data, ref.size); (*secondary_index)[StringRefWrapper(src_column, row)] = next_position; - ++next_position; + + if (auto res = updatePosition(next_position, i)) + return res; } else positions[i] = jt->second; @@ -399,43 +433,92 @@ void ColumnUnique::uniqueInsertRangeImpl( auto ref = src_column->getDataAt(row); column->insertData(ref.data, ref.size); (*index)[StringRefWrapper(column, next_position)] = next_position; - ++next_position; + + if (auto res = updatePosition(next_position, i)) + return res; } } else positions[i] = it->second; } } -} - -template -MutableColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) -{ - auto positions_column = ColumnVector::create(length); - auto & positions = positions_column->getData(); - - uniqueInsertRangeImpl(src, start, length, positions, nullptr, 0); return positions_column; } -template -IColumnUnique::IndexesWithOverflow ColumnUnique::uniqueInsertRangeWithOverflow( +template +MutableColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) +{ + size_t size = getRawColumnPtr()->size(); + + auto callForType = [&](auto x) + { + using IndexType = decltype(x); + if (size <= std::numeric_limits::max()) + { + auto positions_column = ColumnVector::create(length); + auto & positions = positions_column->getData(); + + return uniqueInsertRangeImpl(src, start, length, positions, nullptr, 0); + } + + return nullptr; + }; + + MutableColumnPtr positions_column; + if (!positions_column) + positions_column = callForType(UInt8()); + if (!positions_column) + positions_column = callForType(UInt16()); + if (!positions_column) + positions_column = callForType(UInt32()); + if (!positions_column) + positions_column = callForType(UInt64()); + if (!positions_column) + throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR); + + return positions_column; +} + +template +IColumnUnique::IndexesWithOverflow ColumnUnique::uniqueInsertRangeWithOverflow( const IColumn & src, size_t start, size_t length, size_t max_dictionary_size) { - auto positions_column = ColumnVector::create(length); + size_t size = getRawColumnPtr()->size(); auto overflowed_keys = column_holder->cloneEmpty(); - auto & positions = positions_column->getData(); - auto overflowed_keys_ptr = typeid_cast(overflowed_keys.get()); if (!overflowed_keys_ptr) throw Exception("Invalid keys type for ColumnUnique.", ErrorCodes::LOGICAL_ERROR); - uniqueInsertRangeImpl(src, start, length, positions, overflowed_keys_ptr, max_dictionary_size); + auto callForType = [&](auto x) + { + using IndexType = decltype(x); + if (size <= std::numeric_limits::max()) + { + auto positions_column = ColumnVector::create(length); + auto & positions = positions_column->getData(); + + return uniqueInsertRangeImpl(src, start, length, positions, overflowed_keys_ptr, max_dictionary_size); + } + + return nullptr; + }; + + MutableColumnPtr positions_column; + if (!positions_column) + positions_column = callForType(UInt8()); + if (!positions_column) + positions_column = callForType(UInt16()); + if (!positions_column) + positions_column = callForType(UInt32()); + if (!positions_column) + positions_column = callForType(UInt64()); + if (!positions_column) + throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR); IColumnUnique::IndexesWithOverflow indexes_with_overflow; indexes_with_overflow.indexes = std::move(positions_column); @@ -443,8 +526,8 @@ IColumnUnique::IndexesWithOverflow ColumnUnique::uniqueIn return indexes_with_overflow; } -template -IColumnUnique::SerializableState ColumnUnique::getSerializableState() const +template +IColumnUnique::SerializableState ColumnUnique::getSerializableState() const { IColumnUnique::SerializableState state; state.column = column_holder; diff --git a/dbms/src/Columns/ColumnWithDictionary.cpp b/dbms/src/Columns/ColumnWithDictionary.cpp index 42bfd16dce2..b4489dc5b81 100644 --- a/dbms/src/Columns/ColumnWithDictionary.cpp +++ b/dbms/src/Columns/ColumnWithDictionary.cpp @@ -9,6 +9,8 @@ ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, M { if (!dynamic_cast(column_unique.get())) throw Exception("ColumnUnique expected as argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); + + getSizeOfCurrentIndexType(); } ColumnWithDictionary::ColumnWithDictionary(const ColumnWithDictionary & other) @@ -21,4 +23,90 @@ void ColumnWithDictionary::gather(ColumnGathererStream & gatherer) gatherer.gather(*this); } +MutableColumnPtr ColumnWithDictionary::cloneResized(size_t size) const +{ + auto unique_ptr = column_unique; + return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), indexes->cloneResized(size)); +} + +size_t ColumnWithDictionary::getSizeOfCurrentIndexType() const +{ + if (typeid_cast(indexes.get())) + return sizeof(UInt8); + if (typeid_cast(indexes.get())) + return sizeof(UInt16); + if (typeid_cast(indexes.get())) + return sizeof(UInt32); + if (typeid_cast(indexes.get())) + return sizeof(UInt64); + + throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected ColumnUInt, got " + indexes->getName(), + ErrorCodes::ILLEGAL_COLUMN); +} + +template +void ColumnWithDictionary::convertIndexes() +{ + auto convert = [&](auto x) + { + using CurIndexType = typeof(x); + if (auto * index_col = typeid_cast *>(indexes.get())) + { + if (sizeof(CurIndexType) != sizeof(IndexType)) + { + size_t size = index_col->size(); + auto new_index_col = ColumnVector::create(size); + auto & data = index_col->getData(); + auto & new_data = new_index_col->getData(); + + for (size_t i = 0; i < size; ++i) + new_data[i] = data[i]; + + indexes = std::move(new_index_col); + } + + return true; + } + return false; + }; + + if (!convert(UInt8()) && + !convert(UInt16()) && + !convert(UInt32()) && + !convert(UInt64())) + throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected ColumnUInt, got " + + indexes->getName(), ErrorCodes::ILLEGAL_COLUMN); +} + +void ColumnWithDictionary::insertIndex(size_t value) +{ + auto current_index_type = getSizeOfCurrentIndexType(); + + auto insertForType = [&](auto x) + { + using IndexType = typeof(x); + if (value <= std::numeric_limits::max()) + { + if (sizeof(IndexType) > current_index_type) + convertIndexes(); + + getIndexes()->insert(UInt64(value)); + + return true; + } + return false; + }; + + if (!insertForType(UInt8()) && + !insertForType(UInt16()) && + !insertForType(UInt32()) && + !insertForType(UInt64())) + throw Exception("Unexpected indexes type for ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); +} + +void ColumnWithDictionary::insertIndexesRange(const ColumnPtr & column) +{ + +} + } diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index f363dbe793f..935c4adf156 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -37,34 +37,22 @@ public: std::string getName() const override { return "ColumnWithDictionary"; } const char * getFamilyName() const override { return "ColumnWithDictionary"; } - ColumnPtr convertToFullColumn() const - { - return getUnique()->getNestedColumn()->index(*indexes, 0); - } - + ColumnPtr convertToFullColumn() const { return getUnique()->getNestedColumn()->index(*indexes, 0); } ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); } - MutableColumnPtr cloneResized(size_t size) const override - { - auto unique_ptr = column_unique; - return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), indexes->cloneResized(size)); - } - + MutableColumnPtr cloneResized(size_t size) const override; size_t size() const override { return indexes->size(); } - Field operator[](size_t n) const override { return (*column_unique)[indexes->getUInt(n)]; } void get(size_t n, Field & res) const override { column_unique->get(indexes->getUInt(n), res); } StringRef getDataAt(size_t n) const override { return column_unique->getDataAt(indexes->getUInt(n)); } - StringRef getDataAtWithTerminatingZero(size_t n) const override { return column_unique->getDataAtWithTerminatingZero(indexes->getUInt(n)); } UInt64 get64(size_t n) const override { return column_unique->get64(indexes->getUInt(n)); } - UInt64 getUInt(size_t n) const override { return column_unique->getUInt(indexes->getUInt(n)); } Int64 getInt(size_t n) const override { return column_unique->getInt(indexes->getUInt(n)); } bool isNullAt(size_t n) const override { return column_unique->isNullAt(indexes->getUInt(n)); } @@ -252,6 +240,13 @@ private: ColumnPtr column_unique; ColumnPtr indexes; + size_t getSizeOfCurrentIndexType() const; + + template + void convertIndexes(); + void insertIndex(size_t value); + void insertIndexesRange(const ColumnPtr & column); + }; From 9c763f8090ebb7de942874bcd338bd45f804e612 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Jul 2018 21:19:03 +0300 Subject: [PATCH 109/315] Make DatsTypeWithDictionary independent from index type. --- dbms/src/Columns/ColumnUnique.h | 143 +++-- dbms/src/Columns/ColumnWithDictionary.cpp | 555 ++++++++++++++++-- dbms/src/Columns/ColumnWithDictionary.h | 285 +++++---- dbms/src/Columns/IColumnUnique.h | 17 +- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 414 +++++++------ dbms/src/DataTypes/DataTypeWithDictionary.h | 19 +- dbms/src/DataTypes/IDataType.h | 4 +- dbms/src/Functions/FunctionsConversion.h | 6 +- dbms/src/Functions/FunctionsMiscellaneous.cpp | 47 +- dbms/src/Functions/IFunction.cpp | 18 +- dbms/src/Interpreters/Settings.h | 4 + .../Storages/MergeTree/MergeTreeReader.cpp | 32 +- dbms/src/Storages/MergeTree/MergeTreeReader.h | 1 + .../MergeTree/MergedBlockOutputStream.cpp | 36 +- 14 files changed, 1015 insertions(+), 566 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index e1988723640..fb32c123560 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -39,10 +39,10 @@ struct StringRefWrapper namespace ZeroTraits { template - bool check(const StringRefWrapper x) { return nullptr == x.column; } + bool check(const StringRefWrapper x) { return nullptr == x.column && nullptr == x.ref.data; } template - void set(StringRefWrapper & x) { x.column = nullptr; } + void set(StringRefWrapper & x) { x.column = nullptr; x.ref.data = nullptr; } }; @@ -52,7 +52,7 @@ namespace DB template class ColumnUnique final : public COWPtrHelper> { - friend class COWPtrHelper; + friend class COWPtrHelper>; private: explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable); @@ -60,7 +60,9 @@ private: ColumnUnique(const ColumnUnique & other) : column_holder(other.column_holder), is_nullable(other.is_nullable) {} public: - ColumnPtr getNestedColumn() const override; + MutableColumnPtr cloneEmpty() const override; + + const ColumnPtr & getNestedColumn() const override; const ColumnPtr & getNestedNotNullableColumn() const override { return column_holder; } size_t uniqueInsert(const Field & x) override; @@ -71,7 +73,6 @@ public: size_t uniqueInsertData(const char * pos, size_t length) override; size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) override; size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) override; - IColumnUnique::SerializableState getSerializableState() const override; size_t getDefaultValueIndex() const override { return is_nullable ? 1 : 0; } size_t getNullValueIndex() const override; @@ -119,6 +120,7 @@ public: void forEachSubcolumn(IColumn::ColumnCallback callback) override { callback(column_holder); + index = nullptr; } private: @@ -129,6 +131,7 @@ private: /// For DataTypeNullable, stores null map. mutable ColumnPtr cached_null_mask; + mutable ColumnPtr cached_column_nullable; /// Lazy initialized. std::unique_ptr index; @@ -147,11 +150,19 @@ private: const IColumn & src, size_t start, size_t length, + size_t num_added_rows, typename ColumnVector::MutablePtr && positions_column, ColumnType * overflowed_keys, + IndexMapType * secondary_index, size_t max_dictionary_size); }; +template +MutableColumnPtr ColumnUnique::cloneEmpty() const +{ + return ColumnUnique::create(column_holder->cloneResized(numSpecialValues()), is_nullable); +} + template ColumnUnique::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable()) { @@ -170,7 +181,7 @@ ColumnUnique::ColumnUnique(MutableColumnPtr && holder, bool is_nulla } template -ColumnPtr ColumnUnique::getNestedColumn() const +const ColumnPtr & ColumnUnique::getNestedColumn() const { if (is_nullable) { @@ -180,6 +191,7 @@ ColumnPtr ColumnUnique::getNestedColumn() const ColumnUInt8::MutablePtr null_mask = ColumnUInt8::create(size, UInt8(0)); null_mask->getData()[getNullValueIndex()] = 1; cached_null_mask = std::move(null_mask); + cached_column_nullable = ColumnNullable::create(column_holder, cached_null_mask); } if (cached_null_mask->size() != size) @@ -187,9 +199,10 @@ ColumnPtr ColumnUnique::getNestedColumn() const MutableColumnPtr null_mask = (*std::move(cached_null_mask)).mutate(); static_cast(*null_mask).getData().resize_fill(size); cached_null_mask = std::move(null_mask); + cached_column_nullable = ColumnNullable::create(column_holder, cached_null_mask); } - return ColumnNullable::create(column_holder, cached_null_mask); + return cached_column_nullable; } return column_holder; } @@ -219,7 +232,7 @@ void ColumnUnique::buildIndex() } template -IndexType ColumnUnique::insertIntoMap(const StringRefWrapper & ref, IndexType value) +UInt64 ColumnUnique::insertIntoMap(const StringRefWrapper & ref, UInt64 value) { if (!index) buildIndex(); @@ -242,7 +255,7 @@ size_t ColumnUnique::uniqueInsert(const Field & x) return getNullValueIndex(); auto column = getRawColumnPtr(); - auto prev_size = static_cast(column->size()); + auto prev_size = static_cast(column->size()); if ((*column)[getDefaultValueIndex()] == x) return getDefaultValueIndex(); @@ -261,6 +274,9 @@ size_t ColumnUnique::uniqueInsertFrom(const IColumn & src, size_t n) if (is_nullable && src.isNullAt(n)) return getNullValueIndex(); + if (auto * nullable = typeid_cast(&src)) + return uniqueInsertFrom(nullable->getNestedColumn(), n); + auto ref = src.getDataAt(n); return uniqueInsertData(ref.data, ref.size); } @@ -336,14 +352,31 @@ size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(const char return static_cast(index_pos); } +template +static void checkIndexes(const ColumnVector & indexes, size_t max_dictionary_size) +{ + auto & data = indexes.getData(); + for (size_t i = 0; i < data.size(); ++i) + { + if (data[i] >= max_dictionary_size) + { + throw Exception("Found index " + toString(data[i]) + " at position " + toString(i) + + " which is grated or equal than dictionary size " + toString(max_dictionary_size), + ErrorCodes::LOGICAL_ERROR); + } + } +} + template template MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( const IColumn & src, size_t start, size_t length, + size_t num_added_rows, typename ColumnVector::MutablePtr && positions_column, ColumnType * overflowed_keys, + IndexMapType * secondary_index, size_t max_dictionary_size) { if (!index) @@ -353,9 +386,11 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( const NullMap * null_map = nullptr; auto & positions = positions_column->getData(); - using SuperiorIndexType = NumberTraits::Construct::Type; - auto updatePosition = [&](UInt64 & next_position, UInt64 num_added_rows) -> MutableColumnPtr + auto updatePosition = [&](UInt64 & next_position) -> MutableColumnPtr { + constexpr auto next_size = NumberTraits::nextSize(sizeof(IndexType)); + using SuperiorIndexType = typename NumberTraits::Construct::Type; + ++next_position; if (next_position > std::numeric_limits::max()) @@ -364,102 +399,108 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( throw Exception("Can't find superior index type for type " + demangle(typeid(IndexType).name()), ErrorCodes::LOGICAL_ERROR); - auto expanded_column = ColumnVector::create(length); + auto expanded_column = ColumnVector::create(length); auto & expanded_data = expanded_column->getData(); for (size_t i = 0; i < num_added_rows; ++i) expanded_data[i] = positions[i]; return uniqueInsertRangeImpl( src, - start + num_added_rows, - length - num_added_rows, + start, + length, + num_added_rows, std::move(expanded_column), overflowed_keys, + secondary_index, max_dictionary_size); } return nullptr; }; - if (src.isColumnNullable()) + if (auto nullable_column = typeid_cast(&src)) { - auto nullable_column = static_cast(&src); - src_column = static_cast(&nullable_column->getNestedColumn()); + src_column = typeid_cast(&nullable_column->getNestedColumn()); null_map = &nullable_column->getNullMapData(); } else - src_column = static_cast(&src); + src_column = typeid_cast(&src); - std::unique_ptr secondary_index; - if (overflowed_keys) - secondary_index = std::make_unique(); + if (src_column == nullptr) + throw Exception("Invalid column type for ColumnUnique::insertRangeFrom. Expected " + column_holder->getName() + + ", got " + src.getName(), ErrorCodes::ILLEGAL_COLUMN); auto column = getRawColumnPtr(); UInt64 next_position = column->size(); - for (auto i : ext::range(0, length)) + if (secondary_index) + next_position += secondary_index->size(); + + for (; num_added_rows < length; ++num_added_rows) { - auto row = start + i; + auto row = start + num_added_rows; if (null_map && (*null_map)[row]) - positions[i] = getNullValueIndex(); + positions[num_added_rows] = getNullValueIndex(); else if (column->compareAt(getDefaultValueIndex(), row, *src_column, 1) == 0) - positions[i] = getDefaultValueIndex(); + positions[num_added_rows] = getDefaultValueIndex(); else { auto it = index->find(StringRefWrapper(src_column, row)); if (it == index->end()) { - if (overflowed_keys && next_position >= max_dictionary_size + numSpecialValues()) + if (overflowed_keys && next_position >= max_dictionary_size) { auto jt = secondary_index->find(StringRefWrapper(src_column, row)); if (jt == secondary_index->end()) { - positions[i] = next_position; + positions[num_added_rows] = next_position; auto ref = src_column->getDataAt(row); overflowed_keys->insertData(ref.data, ref.size); (*secondary_index)[StringRefWrapper(src_column, row)] = next_position; - if (auto res = updatePosition(next_position, i)) + if (auto res = updatePosition(next_position)) return res; } else - positions[i] = jt->second; + positions[num_added_rows] = jt->second; } else { - positions[i] = next_position; + positions[num_added_rows] = next_position; auto ref = src_column->getDataAt(row); column->insertData(ref.data, ref.size); (*index)[StringRefWrapper(column, next_position)] = next_position; - if (auto res = updatePosition(next_position, i)) + if (auto res = updatePosition(next_position)) return res; } } else - positions[i] = it->second; + positions[num_added_rows] = it->second; } } - return positions_column; + checkIndexes(*positions_column, column->size() + (overflowed_keys ? overflowed_keys->size() : 0)); + + return std::move(positions_column); } template MutableColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) { - size_t size = getRawColumnPtr()->size(); - auto callForType = [&](auto x) + auto callForType = [this, &src, start, length](auto x) -> MutableColumnPtr { + size_t size = getRawColumnPtr()->size(); + using IndexType = decltype(x); if (size <= std::numeric_limits::max()) { - auto positions_column = ColumnVector::create(length); - auto & positions = positions_column->getData(); - - return uniqueInsertRangeImpl(src, start, length, positions, nullptr, 0); + auto positions = ColumnVector::create(length); + return this->uniqueInsertRangeImpl(src, start, length, 0, + std::move(positions), nullptr, nullptr, 0); } return nullptr; @@ -488,21 +529,22 @@ IColumnUnique::IndexesWithOverflow ColumnUnique::uniqueInsertRangeWi size_t max_dictionary_size) { - size_t size = getRawColumnPtr()->size(); auto overflowed_keys = column_holder->cloneEmpty(); auto overflowed_keys_ptr = typeid_cast(overflowed_keys.get()); if (!overflowed_keys_ptr) throw Exception("Invalid keys type for ColumnUnique.", ErrorCodes::LOGICAL_ERROR); - auto callForType = [&](auto x) + auto callForType = [this, &src, start, length, overflowed_keys_ptr, max_dictionary_size](auto x) -> MutableColumnPtr { + size_t size = getRawColumnPtr()->size(); + using IndexType = decltype(x); if (size <= std::numeric_limits::max()) { - auto positions_column = ColumnVector::create(length); - auto & positions = positions_column->getData(); - - return uniqueInsertRangeImpl(src, start, length, positions, overflowed_keys_ptr, max_dictionary_size); + auto positions = ColumnVector::create(length); + IndexMapType secondary_index; + return this->uniqueInsertRangeImpl(src, start, length, 0, std::move(positions), + overflowed_keys_ptr, &secondary_index, max_dictionary_size); } return nullptr; @@ -526,15 +568,4 @@ IColumnUnique::IndexesWithOverflow ColumnUnique::uniqueInsertRangeWi return indexes_with_overflow; } -template -IColumnUnique::SerializableState ColumnUnique::getSerializableState() const -{ - IColumnUnique::SerializableState state; - state.column = column_holder; - state.offset = numSpecialValues(); - state.limit = column_holder->size() - state.offset; - - return state; -} - }; diff --git a/dbms/src/Columns/ColumnWithDictionary.cpp b/dbms/src/Columns/ColumnWithDictionary.cpp index b4489dc5b81..0f64d4bcbe0 100644 --- a/dbms/src/Columns/ColumnWithDictionary.cpp +++ b/dbms/src/Columns/ColumnWithDictionary.cpp @@ -1,21 +1,192 @@ #include +#include #include +#include +#include namespace DB { -ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_) - : column_unique(std::move(column_unique_)), indexes(std::move(indexes_)) +namespace { - if (!dynamic_cast(column_unique.get())) - throw Exception("ColumnUnique expected as argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); + template + PaddedPODArray * getIndexesData(IColumn & indexes) + { + auto * column = typeid_cast *>(&indexes); + if (column) + return &column->getData(); - getSizeOfCurrentIndexType(); + return nullptr; + } + + template + MutableColumnPtr mapUniqueIndexImpl(PaddedPODArray & index) + { + PaddedPODArray copy(index.cbegin(), index.cend()); + + HashMap hash_map; + for (auto val : index) + hash_map.insert({val, hash_map.size()}); + + auto res_col = ColumnVector::create(); + auto & data = res_col->getData(); + + data.resize(hash_map.size()); + for (auto val : hash_map) + data[val.second] = val.first; + + for (auto & ind : index) + ind = hash_map[ind]; + + for (size_t i = 0; i < index.size(); ++i) + if (data[index[i]] != copy[i]) + throw Exception("Expected " + toString(data[index[i]]) + ", but got " + toString(copy[i]), ErrorCodes::LOGICAL_ERROR); + + return std::move(res_col); + } + + /// Returns unique values of column. Write new index to column. + MutableColumnPtr mapUniqueIndex(IColumn & column) + { + if (auto * data_uint8 = getIndexesData(column)) + return mapUniqueIndexImpl(*data_uint8); + else if (auto * data_uint16 = getIndexesData(column)) + return mapUniqueIndexImpl(*data_uint16); + else if (auto * data_uint32 = getIndexesData(column)) + return mapUniqueIndexImpl(*data_uint32); + else if (auto * data_uint64 = getIndexesData(column)) + return mapUniqueIndexImpl(*data_uint64); + else + throw Exception("Indexes column for getUniqueIndex must be ColumnUInt, got" + column.getName(), + ErrorCodes::LOGICAL_ERROR); + } } -ColumnWithDictionary::ColumnWithDictionary(const ColumnWithDictionary & other) - : column_unique(other.column_unique), indexes(other.indexes) + +ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_) + : dictionary(std::move(column_unique_)), idx(std::move(indexes_)) { + idx.check(getDictionary().size()); +} + +void ColumnWithDictionary::insert(const Field & x) +{ + compactIfSharedDictionary(); + idx.insertPosition(dictionary.getColumnUnique().uniqueInsert(x)); + idx.check(getDictionary().size()); +} + +void ColumnWithDictionary::insertDefault() +{ + idx.insertPosition(getDictionary().getDefaultValueIndex()); +} + +void ColumnWithDictionary::insertFrom(const IColumn & src, size_t n) +{ + auto * src_with_dict = typeid_cast(&src); + + if (!src_with_dict) + throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); + + size_t position = src_with_dict->getIndexes().getUInt(n); + + if (&src_with_dict->getDictionary() == &getDictionary()) + { + /// Dictionary is shared with src column. Insert only index. + idx.insertPosition(position); + } + else + { + compactIfSharedDictionary(); + const auto & nested = *src_with_dict->getDictionary().getNestedColumn(); + idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(nested, position)); + } + + idx.check(getDictionary().size()); +} + +void ColumnWithDictionary::insertFromFullColumn(const IColumn & src, size_t n) +{ + compactIfSharedDictionary(); + idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(src, n)); + idx.check(getDictionary().size()); +} + +void ColumnWithDictionary::insertRangeFrom(const IColumn & src, size_t start, size_t length) +{ + auto * src_with_dict = typeid_cast(&src); + + if (!src_with_dict) + throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); + + if (&src_with_dict->getDictionary() == &getDictionary()) + { + /// Dictionary is shared with src column. Insert only indexes. + idx.insertPositionsRange(src_with_dict->getIndexes(), start, length); + } + else + { + compactIfSharedDictionary(); + + /// TODO: Support native insertion from other unique column. It will help to avoid null map creation. + + auto sub_idx = (*src_with_dict->getIndexes().cut(start, length)).mutate(); + auto idx_map = mapUniqueIndex(*sub_idx); + + auto src_nested = src_with_dict->getDictionary().getNestedColumn(); + auto used_keys = src_nested->index(*idx_map, 0); + + auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(*used_keys, 0, used_keys->size()); + idx.insertPositionsRange(*inserted_indexes->index(*sub_idx, 0), 0, length); + } + idx.check(getDictionary().size()); +} + +void ColumnWithDictionary::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length) +{ + compactIfSharedDictionary(); + auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(src, start, length); + idx.insertPositionsRange(*inserted_indexes, 0, length); + idx.check(getDictionary().size()); +} + +void ColumnWithDictionary::insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions) +{ + Index(positions.getPtr()).check(keys.size()); + compactIfSharedDictionary(); + auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(keys, 0, keys.size()); + idx.insertPositionsRange(*inserted_indexes->index(positions, 0), 0, positions.size()); + idx.check(getDictionary().size()); +} + +void ColumnWithDictionary::insertData(const char * pos, size_t length) +{ + compactIfSharedDictionary(); + idx.insertPosition(dictionary.getColumnUnique().uniqueInsertData(pos, length)); + idx.check(getDictionary().size()); +} + +void ColumnWithDictionary::insertDataWithTerminatingZero(const char * pos, size_t length) +{ + compactIfSharedDictionary(); + idx.insertPosition(dictionary.getColumnUnique().uniqueInsertDataWithTerminatingZero(pos, length)); + idx.check(getDictionary().size()); +} + +StringRef ColumnWithDictionary::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +{ + return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin); +} + +const char * ColumnWithDictionary::deserializeAndInsertFromArena(const char * pos) +{ + compactIfSharedDictionary(); + + const char * new_pos; + idx.insertPosition(dictionary.getColumnUnique().uniqueDeserializeAndInsertFromArena(pos, new_pos)); + + idx.check(getDictionary().size()); + return new_pos; } void ColumnWithDictionary::gather(ColumnGathererStream & gatherer) @@ -25,88 +196,364 @@ void ColumnWithDictionary::gather(ColumnGathererStream & gatherer) MutableColumnPtr ColumnWithDictionary::cloneResized(size_t size) const { - auto unique_ptr = column_unique; - return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), indexes->cloneResized(size)); + auto unique_ptr = dictionary.getColumnUniquePtr(); + return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), getIndexes().cloneResized(size)); } -size_t ColumnWithDictionary::getSizeOfCurrentIndexType() const +int ColumnWithDictionary::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { - if (typeid_cast(indexes.get())) - return sizeof(UInt8); - if (typeid_cast(indexes.get())) - return sizeof(UInt16); - if (typeid_cast(indexes.get())) - return sizeof(UInt32); - if (typeid_cast(indexes.get())) - return sizeof(UInt64); + const auto & column_with_dictionary = static_cast(rhs); + size_t n_index = getIndexes().getUInt(n); + size_t m_index = column_with_dictionary.getIndexes().getUInt(m); + return getDictionary().compareAt(n_index, m_index, column_with_dictionary.getDictionary(), nan_direction_hint); +} - throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected ColumnUInt, got " + indexes->getName(), +void ColumnWithDictionary::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + if (limit == 0) + limit = size(); + + size_t unique_limit = std::min(limit, getDictionary().size()); + Permutation unique_perm; + getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); + + /// TODO: optimize with sse. + + /// Get indexes per row in column_unique. + std::vector> indexes_per_row(getDictionary().size()); + size_t indexes_size = getIndexes().size(); + for (size_t row = 0; row < indexes_size; ++row) + indexes_per_row[getIndexes().getUInt(row)].push_back(row); + + /// Replicate permutation. + size_t perm_size = std::min(indexes_size, limit); + res.resize(perm_size); + size_t perm_index = 0; + for (size_t row = 0; row < indexes_size && perm_index < perm_size; ++row) + { + const auto & row_indexes = indexes_per_row[unique_perm[row]]; + for (auto row_index : row_indexes) + { + res[perm_index] = row_index; + ++perm_index; + + if (perm_index == perm_size) + break; + } + } +} + +std::vector ColumnWithDictionary::scatter(ColumnIndex num_columns, const Selector & selector) const +{ + auto columns = getIndexes().scatter(num_columns, selector); + for (auto & column : columns) + { + auto unique_ptr = dictionary.getColumnUniquePtr(); + column = ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), std::move(column)); + } + + return columns; +} + +void ColumnWithDictionary::setSharedDictionary(const ColumnPtr & column_unique) +{ + if (!empty()) + throw Exception("Can't set ColumnUnique for ColumnWithDictionary because is't not empty.", + ErrorCodes::LOGICAL_ERROR); + + dictionary.setShared(column_unique); +} + +ColumnWithDictionary::MutablePtr ColumnWithDictionary::compact() +{ + auto positions = idx.getPositions(); + /// Create column with new indexes and old dictionary. + auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), (*std::move(positions)).mutate()); + /// Will create new dictionary. + column->compactInplace(); + + return column; +} + +ColumnWithDictionary::MutablePtr ColumnWithDictionary::cutAndCompact(size_t start, size_t length) const +{ + auto sub_positions = (*idx.getPositions()->cut(start, length)).mutate(); + /// Create column with new indexes and old dictionary. + auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), std::move(sub_positions)); + /// Will create new dictionary. + column->compactInplace(); + + return column; +} + +void ColumnWithDictionary::compactInplace() +{ + auto positions = idx.detachPositions(); + dictionary.compact(positions); + idx.attachPositions(std::move(positions)); +} + +void ColumnWithDictionary::compactIfSharedDictionary() +{ + if (dictionary.isShared()) + compactInplace(); +} + + +ColumnWithDictionary::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {} + +ColumnWithDictionary::Index::Index(MutableColumnPtr && positions) : positions(std::move(positions)) +{ + updateSizeOfType(); +} + +ColumnWithDictionary::Index::Index(ColumnPtr positions) : positions(std::move(positions)) +{ + updateSizeOfType(); +} + +template +void ColumnWithDictionary::Index::callForType(Callback && callback, size_t size_of_type) +{ + switch (size_of_type) + { + case sizeof(UInt8): { callback(UInt8()); break; } + case sizeof(UInt16): { callback(UInt16()); break; } + case sizeof(UInt32): { callback(UInt32()); break; } + case sizeof(UInt64): { callback(UInt64()); break; } + default: { + throw Exception("Unexpected size of index type for ColumnWithDictionary: " + toString(size_of_type), + ErrorCodes::LOGICAL_ERROR); + } + } +} + +size_t ColumnWithDictionary::Index::getSizeOfIndexType(const IColumn & column, size_t hint) +{ + auto checkFor = [&](auto type) { return typeid_cast *>(&column) != nullptr; }; + auto tryGetSizeFor = [&](auto type) -> size_t { return checkFor(type) ? sizeof(decltype(type)) : 0; }; + + if (hint) + { + size_t size = 0; + callForType([&](auto type) { size = tryGetSizeFor(type); }, hint); + + if (size) + return size; + } + + if (auto size = tryGetSizeFor(UInt8())) + return size; + if (auto size = tryGetSizeFor(UInt16())) + return size; + if (auto size = tryGetSizeFor(UInt32())) + return size; + if (auto size = tryGetSizeFor(UInt64())) + return size; + + throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected UInt, got " + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } +void ColumnWithDictionary::Index::attachPositions(ColumnPtr positions_) +{ + positions = std::move(positions_); + updateSizeOfType(); +} + template -void ColumnWithDictionary::convertIndexes() +typename ColumnVector::Container & ColumnWithDictionary::Index::getPositionsData() +{ + auto * positions_ptr = typeid_cast *>(positions->assumeMutable().get()); + if (!positions_ptr) + throw Exception("Invalid indexes type for ColumnWithDictionary." + " Expected UInt" + toString(8 * sizeof(IndexType)) + ", got " + positions->getName(), + ErrorCodes::LOGICAL_ERROR); + + return positions_ptr->getData(); +} + +template +void ColumnWithDictionary::Index::convertPositions() { auto convert = [&](auto x) { - using CurIndexType = typeof(x); - if (auto * index_col = typeid_cast *>(indexes.get())) + using CurIndexType = decltype(x); + auto & data = getPositionsData(); + + if (sizeof(CurIndexType) > sizeof(IndexType)) + throw Exception("Converting indexes to smaller type: from " + toString(sizeof(CurIndexType)) + + " to " + toString(sizeof(IndexType)), ErrorCodes::LOGICAL_ERROR); + + if (sizeof(CurIndexType) != sizeof(IndexType)) { - if (sizeof(CurIndexType) != sizeof(IndexType)) - { - size_t size = index_col->size(); - auto new_index_col = ColumnVector::create(size); - auto & data = index_col->getData(); - auto & new_data = new_index_col->getData(); + size_t size = data.size(); + auto new_positions = ColumnVector::create(size); + auto & new_data = new_positions->getData(); - for (size_t i = 0; i < size; ++i) - new_data[i] = data[i]; + /// TODO: Optimize with SSE? + for (size_t i = 0; i < size; ++i) + new_data[i] = data[i]; - indexes = std::move(new_index_col); - } - - return true; + positions = std::move(new_positions); + size_of_type = sizeof(IndexType); } - return false; }; - if (!convert(UInt8()) && - !convert(UInt16()) && - !convert(UInt32()) && - !convert(UInt64())) - throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected ColumnUInt, got " - + indexes->getName(), ErrorCodes::ILLEGAL_COLUMN); + callForType(std::move(convert), size_of_type); + + checkSizeOfType(); } -void ColumnWithDictionary::insertIndex(size_t value) +void ColumnWithDictionary::Index::expandType() { - auto current_index_type = getSizeOfCurrentIndexType(); - - auto insertForType = [&](auto x) + auto expand = [&](auto type) { - using IndexType = typeof(x); - if (value <= std::numeric_limits::max()) + using CurIndexType = decltype(type); + constexpr auto next_size = NumberTraits::nextSize(sizeof(CurIndexType)); + if (next_size == sizeof(CurIndexType)) + throw Exception("Can't expand indexes type for ColumnWithDictionary from type: " + + demangle(typeid(CurIndexType).name()), ErrorCodes::LOGICAL_ERROR); + + using NewIndexType = typename NumberTraits::Construct::Type; + convertPositions(); + }; + + callForType(std::move(expand), size_of_type); +} + +UInt64 ColumnWithDictionary::Index::getMaxPositionForCurrentType() const +{ + UInt64 value = 0; + callForType([&](auto type) { value = std::numeric_limits::max(); }, size_of_type); + return value; +} + +void ColumnWithDictionary::Index::insertPosition(UInt64 position) +{ + while (position > getMaxPositionForCurrentType()) + expandType(); + + positions->assumeMutableRef().insert(UInt64(position)); + checkSizeOfType(); +} + +void ColumnWithDictionary::Index::insertPositionsRange(const IColumn & column, size_t offset, size_t limit) +{ + auto insertForType = [&](auto type) + { + using ColumnType = decltype(type); + const auto * column_ptr = typeid_cast *>(&column); + + if (!column_ptr) + return false; + + if (size_of_type < sizeof(ColumnType)) + convertPositions(); + + if (size_of_type == sizeof(ColumnType)) + positions->assumeMutableRef().insertRangeFrom(column, offset, limit); + else { - if (sizeof(IndexType) > current_index_type) - convertIndexes(); + auto copy = [&](auto cur_type) + { + using CurIndexType = decltype(cur_type); + auto & positions_data = getPositionsData(); + const auto & column_data = column_ptr->getData(); - getIndexes()->insert(UInt64(value)); + size_t size = positions_data.size(); + positions_data.resize(size + limit); - return true; + for (size_t i = 0; i < limit; ++i) + positions_data[size + i] = column_data[offset + i]; + }; + + callForType(std::move(copy), size_of_type); } - return false; + + return true; }; if (!insertForType(UInt8()) && !insertForType(UInt16()) && !insertForType(UInt32()) && !insertForType(UInt64())) - throw Exception("Unexpected indexes type for ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Invalid column for ColumnWithDictionary index. Expected UInt, got " + column.getName(), + ErrorCodes::ILLEGAL_COLUMN); + + checkSizeOfType(); } -void ColumnWithDictionary::insertIndexesRange(const ColumnPtr & column) +void ColumnWithDictionary::Index::check(size_t max_dictionary_size) +{ + auto check = [&](auto cur_type) + { + using CurIndexType = decltype(cur_type); + auto & positions_data = getPositionsData(); + + for (size_t i = 0; i < positions_data.size(); ++i) + { + if (positions_data[i] >= max_dictionary_size) + { + throw Exception("Found index " + toString(positions_data[i]) + " at position " + toString(i) + + " which is grated or equal than dictionary size " + toString(max_dictionary_size), + ErrorCodes::LOGICAL_ERROR); + } + } + }; + + callForType(std::move(check), size_of_type); +} + +void ColumnWithDictionary::Index::checkSizeOfType() +{ + if (size_of_type != getSizeOfIndexType(*positions, size_of_type)) + throw Exception("Invalid size of type. Expected " + toString(8 * size_of_type) + + ", but positions are " + positions->getName(), ErrorCodes::LOGICAL_ERROR); +} + + +ColumnWithDictionary::Dictionary::Dictionary(MutableColumnPtr && column_unique_) + : column_unique(std::move(column_unique_)) +{ + checkColumn(*column_unique); +} +ColumnWithDictionary::Dictionary::Dictionary(ColumnPtr column_unique_) + : column_unique(std::move(column_unique_)) +{ + checkColumn(*column_unique); +} + +void ColumnWithDictionary::Dictionary::checkColumn(const IColumn & column) { + if (!dynamic_cast(&column)) + throw Exception("ColumnUnique expected as an argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); +} + +void ColumnWithDictionary::Dictionary::setShared(const ColumnPtr & dictionary) +{ + checkColumn(*dictionary); + + column_unique = dictionary; + shared = true; +} + +void ColumnWithDictionary::Dictionary::compact(ColumnPtr & positions) +{ + auto new_column_unique = column_unique->cloneEmpty(); + + auto & unique = getColumnUnique(); + auto & new_unique = static_cast(*new_column_unique); + + auto indexes = mapUniqueIndex(positions->assumeMutableRef()); + auto sub_keys = unique.getNestedColumn()->index(*indexes, 0); + auto new_indexes = new_unique.uniqueInsertRangeFrom(*sub_keys, 0, sub_keys->size()); + + positions = (*new_indexes->index(*positions, 0)).mutate(); + column_unique = std::move(new_column_unique); + + shared = false; } } diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index 935c4adf156..aaced565e14 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -18,7 +18,7 @@ class ColumnWithDictionary final : public COWPtrHelper; ColumnWithDictionary(MutableColumnPtr && column_unique, MutableColumnPtr && indexes); - ColumnWithDictionary(const ColumnWithDictionary & other); + ColumnWithDictionary(const ColumnWithDictionary & other) = default; public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. @@ -37,216 +37,203 @@ public: std::string getName() const override { return "ColumnWithDictionary"; } const char * getFamilyName() const override { return "ColumnWithDictionary"; } - ColumnPtr convertToFullColumn() const { return getUnique()->getNestedColumn()->index(*indexes, 0); } + ColumnPtr convertToFullColumn() const { return getDictionary().getNestedColumn()->index(getIndexes(), 0); } ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); } MutableColumnPtr cloneResized(size_t size) const override; - size_t size() const override { return indexes->size(); } + size_t size() const override { return getIndexes().size(); } - Field operator[](size_t n) const override { return (*column_unique)[indexes->getUInt(n)]; } - void get(size_t n, Field & res) const override { column_unique->get(indexes->getUInt(n), res); } + Field operator[](size_t n) const override { return getDictionary()[getIndexes().getUInt(n)]; } + void get(size_t n, Field & res) const override { getDictionary().get(getIndexes().getUInt(n), res); } - StringRef getDataAt(size_t n) const override { return column_unique->getDataAt(indexes->getUInt(n)); } + StringRef getDataAt(size_t n) const override { return getDictionary().getDataAt(getIndexes().getUInt(n)); } StringRef getDataAtWithTerminatingZero(size_t n) const override { - return column_unique->getDataAtWithTerminatingZero(indexes->getUInt(n)); + return getDictionary().getDataAtWithTerminatingZero(getIndexes().getUInt(n)); } - UInt64 get64(size_t n) const override { return column_unique->get64(indexes->getUInt(n)); } - UInt64 getUInt(size_t n) const override { return column_unique->getUInt(indexes->getUInt(n)); } - Int64 getInt(size_t n) const override { return column_unique->getInt(indexes->getUInt(n)); } - bool isNullAt(size_t n) const override { return column_unique->isNullAt(indexes->getUInt(n)); } + UInt64 get64(size_t n) const override { return getDictionary().get64(getIndexes().getUInt(n)); } + UInt64 getUInt(size_t n) const override { return getDictionary().getUInt(getIndexes().getUInt(n)); } + Int64 getInt(size_t n) const override { return getDictionary().getInt(getIndexes().getUInt(n)); } + bool isNullAt(size_t n) const override { return getDictionary().isNullAt(getIndexes().getUInt(n)); } ColumnPtr cut(size_t start, size_t length) const override { - return ColumnWithDictionary::create(column_unique, indexes->cut(start, length)); + return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().cut(start, length)); } - void insert(const Field & x) override { getIndexes()->insert(Field(UInt64(getUnique()->uniqueInsert(x)))); } + void insert(const Field & x) override; + void insertDefault() override; - void insertFromFullColumn(const IColumn & src, size_t n) - { - getIndexes()->insert(getUnique()->uniqueInsertFrom(src, n)); - } - void insertFrom(const IColumn & src, size_t n) override - { - if (!typeid_cast(&src)) - throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); + void insertFrom(const IColumn & src, size_t n) override; + void insertFromFullColumn(const IColumn & src, size_t n); - auto & src_with_dict = static_cast(src); - size_t idx = src_with_dict.getIndexes()->getUInt(n); - insertFromFullColumn(*src_with_dict.getUnique()->getNestedColumn(), idx); - } + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length); + void insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions); - void insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length) - { - auto inserted_indexes = getUnique()->uniqueInsertRangeFrom(src, start, length); - getIndexes()->insertRangeFrom(*inserted_indexes, 0, length); - } - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override - { - if (!typeid_cast(&src)) - throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); + void insertData(const char * pos, size_t length) override; + void insertDataWithTerminatingZero(const char * pos, size_t length) override; - auto & src_with_dict = static_cast(src); - /// TODO: Support native insertion from other unique column. It will help to avoid null map creation. - auto src_nested = src_with_dict.getUnique()->getNestedColumn(); - auto inserted_idx = getUnique()->uniqueInsertRangeFrom(*src_nested, 0, src_nested->size()); - auto idx = inserted_idx->index(*src_with_dict.getIndexes()->cut(start, length), 0); - getIndexes()->insertRangeFrom(*idx, 0, length); - } - void insertData(const char * pos, size_t length) override - { - getIndexes()->insert(Field(UInt64(getUnique()->uniqueInsertData(pos, length)))); - } + void popBack(size_t n) override { idx.popBack(n); } - void insertDataWithTerminatingZero(const char * pos, size_t length) override - { - getIndexes()->insert(Field(UInt64(getUnique()->uniqueInsertDataWithTerminatingZero(pos, length)))); - } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - void insertDefault() override - { - getIndexes()->insert(getUnique()->getDefaultValueIndex()); - } - - void popBack(size_t n) override { getIndexes()->popBack(n); } - - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override - { - return getUnique()->serializeValueIntoArena(indexes->getUInt(n), arena, begin); - } - - const char * deserializeAndInsertFromArena(const char * pos) override - { - const char * new_pos; - getIndexes()->insert(getUnique()->uniqueDeserializeAndInsertFromArena(pos, new_pos)); - return new_pos; - } + const char * deserializeAndInsertFromArena(const char * pos) override; void updateHashWithValue(size_t n, SipHash & hash) const override { - return getUnique()->updateHashWithValue(indexes->getUInt(n), hash); + return getDictionary().updateHashWithValue(getIndexes().getUInt(n), hash); } ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { - return ColumnWithDictionary::create(column_unique, indexes->filter(filt, result_size_hint)); + return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint)); } ColumnPtr permute(const Permutation & perm, size_t limit) const override { - return ColumnWithDictionary::create(column_unique, indexes->permute(perm, limit)); + return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().permute(perm, limit)); } ColumnPtr index(const IColumn & indexes_, size_t limit) const override { - return ColumnWithDictionary::create(column_unique, indexes->index(indexes_, limit)); + return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().index(indexes_, limit)); } - int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override - { - const auto & column_with_dictionary = static_cast(rhs); - size_t n_index = indexes->getUInt(n); - size_t m_index = column_with_dictionary.indexes->getUInt(m); - return getUnique()->compareAt(n_index, m_index, *column_with_dictionary.column_unique, nan_direction_hint); - } + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; - void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override - { - if (limit == 0) - limit = size(); - - size_t unique_limit = std::min(limit, getUnique()->size()); - Permutation unique_perm; - getUnique()->getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); - - /// TODO: optimize with sse. - - /// Get indexes per row in column_unique. - std::vector> indexes_per_row(getUnique()->size()); - size_t indexes_size = indexes->size(); - for (size_t row = 0; row < indexes_size; ++row) - indexes_per_row[indexes->getUInt(row)].push_back(row); - - /// Replicate permutation. - size_t perm_size = std::min(indexes_size, limit); - res.resize(perm_size); - size_t perm_index = 0; - for (size_t row = 0; row < indexes_size && perm_index < perm_size; ++row) - { - const auto & row_indexes = indexes_per_row[unique_perm[row]]; - for (auto row_index : row_indexes) - { - res[perm_index] = row_index; - ++perm_index; - - if (perm_index == perm_size) - break; - } - } - } + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; ColumnPtr replicate(const Offsets & offsets) const override { - return ColumnWithDictionary::create(column_unique, indexes->replicate(offsets)); + return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets)); } - std::vector scatter(ColumnIndex num_columns, const Selector & selector) const override - { - auto columns = indexes->scatter(num_columns, selector); - for (auto & column : columns) - { - auto unique_ptr = column_unique; - column = ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), std::move(column)); - } - - return columns; - } + std::vector scatter(ColumnIndex num_columns, const Selector & selector) const override; void gather(ColumnGathererStream & gatherer_stream) override ; - void getExtremes(Field & min, Field & max) const override { return column_unique->getExtremes(min, max); } + void getExtremes(Field & min, Field & max) const override { + return getDictionary().index(getIndexes(), 0)->getExtremes(min, max); /// TODO: optimize + } - void reserve(size_t n) override { getIndexes()->reserve(n); } + void reserve(size_t n) override { idx.reserve(n); } - size_t byteSize() const override { return indexes->byteSize() + column_unique->byteSize(); } - size_t allocatedBytes() const override { return indexes->allocatedBytes() + column_unique->allocatedBytes(); } + size_t byteSize() const override { return idx.getPositions()->byteSize() + getDictionary().byteSize(); } + size_t allocatedBytes() const override { return idx.getPositions()->allocatedBytes() + getDictionary().allocatedBytes(); } void forEachSubcolumn(ColumnCallback callback) override { - callback(column_unique); - callback(indexes); + callback(idx.getPositionsPtr()); + + /// Column doesn't own dictionary if it's shared. + if (!dictionary.isShared()) + callback(dictionary.getColumnUniquePtr()); } - bool valuesHaveFixedSize() const override { return column_unique->valuesHaveFixedSize(); } - bool isFixedAndContiguous() const override { return column_unique->isFixedAndContiguous(); } - size_t sizeOfValueIfFixed() const override { return column_unique->sizeOfValueIfFixed(); } - bool isNumeric() const override { return column_unique->isNumeric(); } + bool valuesHaveFixedSize() const override { return getDictionary().valuesHaveFixedSize(); } + bool isFixedAndContiguous() const override { return getDictionary().isFixedAndContiguous(); } + size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); } + bool isNumeric() const override { return getDictionary().isNumeric(); } - IColumnUnique * getUnique() { return static_cast(column_unique->assumeMutable().get()); } - const IColumnUnique * getUnique() const { return static_cast(column_unique->assumeMutable().get()); } - ColumnPtr getUniquePtr() const { return column_unique; } + const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); } + /// IColumnUnique & getUnique() { return static_cast(*column_unique->assumeMutable()); } + /// ColumnPtr getUniquePtr() const { return column_unique; } - IColumn * getIndexes() { return indexes->assumeMutable().get(); } - const IColumn * getIndexes() const { return indexes.get(); } - const ColumnPtr & getIndexesPtr() const { return indexes; } + /// IColumn & getIndexes() { return idx.getPositions()->assumeMutableRef(); } + const IColumn & getIndexes() const { return *idx.getPositions(); } + const ColumnPtr & getIndexesPtr() const { return idx.getPositions(); } - void setIndexes(MutableColumnPtr && indexes_) { indexes = std::move(indexes_); } - void setUnique(const ColumnPtr & unique) { column_unique = unique; } + ///void setIndexes(MutableColumnPtr && indexes_) { indexes = std::move(indexes_); } + + /// Set shared ColumnUnique for empty column with dictionary. + void setSharedDictionary(const ColumnPtr & column_unique); + + /// Create column new dictionary with only keys that are mentioned in index. + MutablePtr compact(); + + /// Cut + compact. + MutablePtr cutAndCompact(size_t start, size_t length) const; bool withDictionary() const override { return true; } + class Index + { + public: + Index(); + Index(const Index & other) = default; + explicit Index(MutableColumnPtr && positions); + explicit Index(ColumnPtr positions); + + const ColumnPtr & getPositions() const { return positions; } + ColumnPtr & getPositionsPtr() { return positions; } + void insertPosition(UInt64 position); + void insertPositionsRange(const IColumn & column, size_t offset, size_t limit); + + void popBack(size_t n) { positions->assumeMutableRef().popBack(n); } + void reserve(size_t n) { positions->assumeMutableRef().reserve(n); } + + UInt64 getMaxPositionForCurrentType() const; + + static size_t getSizeOfIndexType(const IColumn & column, size_t hint); + + void check(size_t max_dictionary_size); + void checkSizeOfType(); + + ColumnPtr detachPositions() { return std::move(positions); } + void attachPositions(ColumnPtr positions_); + + private: + ColumnPtr positions; + size_t size_of_type = 0; + + void updateSizeOfType() { size_of_type = getSizeOfIndexType(*positions, size_of_type); } + void expandType(); + + template + typename ColumnVector::Container & getPositionsData(); + + template + void convertPositions(); + + template + static void callForType(Callback && callback, size_t size_of_type); + }; + private: - ColumnPtr column_unique; - ColumnPtr indexes; + class Dictionary + { + public: + Dictionary(const Dictionary & other) = default; + explicit Dictionary(MutableColumnPtr && column_unique); + explicit Dictionary(ColumnPtr column_unique); - size_t getSizeOfCurrentIndexType() const; + const ColumnPtr & getColumnUniquePtr() const { return column_unique; } + ColumnPtr & getColumnUniquePtr() { return column_unique; } - template - void convertIndexes(); - void insertIndex(size_t value); - void insertIndexesRange(const ColumnPtr & column); + const IColumnUnique & getColumnUnique() const { return static_cast(*column_unique); } + IColumnUnique & getColumnUnique() { return static_cast(column_unique->assumeMutableRef()); } + /// Dictionary may be shared for several mutable columns. + /// Immutable columns may have the same column unique, which isn't necessarily shared dictionary. + void setShared(const ColumnPtr & dictionary); + bool isShared() const { return shared; } + + /// Create new dictionary with only keys that are mentioned in positions. + void compact(ColumnPtr & positions); + + private: + ColumnPtr column_unique; + bool shared = false; + + void checkColumn(const IColumn & column); + }; + + Dictionary dictionary; + Index idx; + + void compactInplace(); + void compactIfSharedDictionary(); }; diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index aedaf3fed1f..44d655f2aab 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -12,7 +12,7 @@ public: /// Column always contains Null if it's Nullable and empty string if it's String or Nullable(String). /// So, size may be greater than the number of inserted unique values. - virtual ColumnPtr getNestedColumn() const = 0; + virtual const ColumnPtr & getNestedColumn() const = 0; /// The same as getNestedColumn, but removes null map if nested column is nullable. virtual const ColumnPtr & getNestedNotNullableColumn() const = 0; @@ -50,21 +50,6 @@ public: virtual size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) = 0; - /// Column which contains the set of necessary for serialization keys. Such that empty column after - /// uniqueInsertRangeFrom(column->cut(offset, limit), 0, limit) call will contain the same set of keys. - struct SerializableState - { - ColumnPtr column; - size_t offset; - size_t limit; - }; - - virtual SerializableState getSerializableState() const = 0; - -// virtual MutableColumnPtr getInsertionPoints(const ColumnPtr & keys) const = 0; -// -// virtual bool has(const char * pos, size_t length) const { return getInsertionPoint(pos, length) != size(); } - const char * getFamilyName() const override { return "ColumnUnique"; } void insert(const Field &) override diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 178404a7368..5259718c9b0 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -34,13 +34,9 @@ namespace } } -DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_, DataTypePtr indexes_type_) - : dictionary_type(std::move(dictionary_type_)), indexes_type(std::move(indexes_type_)) +DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_) + : dictionary_type(std::move(dictionary_type_)) { - if (!indexes_type->isUnsignedInteger()) - throw Exception("Index type of DataTypeWithDictionary must be unsigned integer, but got " - + indexes_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - auto inner_type = dictionary_type; if (dictionary_type->isNullable()) inner_type = static_cast(*dictionary_type).getNestedType(); @@ -57,7 +53,7 @@ void DataTypeWithDictionary::enumerateStreams(const StreamCallback & callback, S path.push_back(Substream::DictionaryKeys); dictionary_type->enumerateStreams(callback, path); path.back() = Substream::DictionaryIndexes; - indexes_type->enumerateStreams(callback, path); + callback(path); path.pop_back(); } @@ -147,20 +143,20 @@ struct IndexesSerializationType type = static_cast(resetFlags(val)); } - IndexesSerializationType(const IDataType & data_type, bool has_additional_keys, bool need_global_dictionary) + IndexesSerializationType(const IColumn & column, bool has_additional_keys, bool need_global_dictionary) : has_additional_keys(has_additional_keys), need_global_dictionary(need_global_dictionary) { - if (typeid_cast(&data_type)) + if (typeid_cast(&column)) type = TUInt8; - else if (typeid_cast(&data_type)) + else if (typeid_cast(&column)) type = TUInt16; - else if (typeid_cast(&data_type)) + else if (typeid_cast(&column)) type = TUInt32; - else if (typeid_cast(&data_type)) + else if (typeid_cast(&column)) type = TUInt64; else - throw Exception("Invalid DataType for IndexesSerializationType. Expected UInt*, got " + data_type.getName(), - ErrorCodes::LOGICAL_ERROR); + throw Exception("Invalid Indexes column for IndexesSerializationType. Expected ColumnUInt*, got " + + column.getName(), ErrorCodes::LOGICAL_ERROR); } DataTypePtr getDataType() const @@ -196,10 +192,9 @@ struct DeserializeStateWithDictionary : public IDataType::DeserializeBinaryBulkS { KeysSerializationVersion key_version; ColumnUniquePtr global_dictionary; - UInt64 num_bytes_in_dictionary; IndexesSerializationType index_type; - MutableColumnPtr additional_keys; + ColumnPtr additional_keys; UInt64 num_pending_rows = 0; explicit DeserializeStateWithDictionary(UInt64 key_version) : key_version(key_version) {} @@ -252,7 +247,7 @@ void DataTypeWithDictionary::serializeBinaryBulkStatePrefix( writeIntBinary(key_version, *stream); - auto column_unique = createColumnUnique(*dictionary_type, *indexes_type); + auto column_unique = createColumnUnique(*dictionary_type); state = std::make_shared(key_version, std::move(column_unique)); } @@ -263,24 +258,21 @@ void DataTypeWithDictionary::serializeBinaryBulkStateSuffix( auto * state_with_dictionary = checkAndGetWithDictionarySerializeState(state); KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value); - if (state_with_dictionary->global_dictionary) + if (state_with_dictionary->global_dictionary && settings.max_dictionary_size) { - auto unique_state = state_with_dictionary->global_dictionary->getSerializableState(); - UInt64 num_keys = unique_state.limit; - if (settings.max_dictionary_size) - { - settings.path.push_back(Substream::DictionaryKeys); - auto * stream = settings.getter(settings.path); - settings.path.pop_back(); + auto nested_column = state_with_dictionary->global_dictionary->getNestedNotNullableColumn(); - if (!stream) - throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStateSuffix", - ErrorCodes::LOGICAL_ERROR); + settings.path.push_back(Substream::DictionaryKeys); + auto * stream = settings.getter(settings.path); + settings.path.pop_back(); - writeIntBinary(num_keys, *stream); - removeNullable(dictionary_type)->serializeBinaryBulk(*unique_state.column, *stream, - unique_state.offset, unique_state.limit); - } + if (!stream) + throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStateSuffix", + ErrorCodes::LOGICAL_ERROR); + + UInt64 num_keys = nested_column->size(); + writeIntBinary(num_keys, *stream); + removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *stream, 0, num_keys); } } @@ -314,79 +306,76 @@ namespace return nullptr; } - template - MutableColumnPtr mapUniqueIndexImpl(PaddedPODArray & index) + struct IndexMapsWithAdditionalKeys { - HashMap hash_map; - for (auto val : index) - hash_map.insert({val, hash_map.size()}); - - auto res_col = ColumnVector::create(); - auto & data = res_col->getData(); - - data.resize(hash_map.size()); - for (auto val : hash_map) - data[val.second] = val.first; - - for (auto & ind : index) - ind = hash_map[ind]; - - return std::move(res_col); - } - - /// Returns unique values of column. Write new index to column. - MutableColumnPtr mapUniqueIndex(IColumn & column) - { - if (auto * data_uint8 = getIndexesData(column)) - return mapUniqueIndexImpl(*data_uint8); - else if (auto * data_uint16 = getIndexesData(column)) - return mapUniqueIndexImpl(*data_uint16); - else if (auto * data_uint32 = getIndexesData(column)) - return mapUniqueIndexImpl(*data_uint32); - else if (auto * data_uint64 = getIndexesData(column)) - return mapUniqueIndexImpl(*data_uint64); - else - throw Exception("Indexes column for getUniqueIndex must be ColumnUInt, got" + column.getName(), - ErrorCodes::LOGICAL_ERROR); - } + MutableColumnPtr dictionary_map; + MutableColumnPtr additional_keys_map; + }; template - MutableColumnPtr mapIndexWithOverflow(PaddedPODArray & index, size_t max_val) + IndexMapsWithAdditionalKeys mapIndexWithAdditionalKeys(PaddedPODArray & index, size_t dict_size) { - HashMap hash_map; + PaddedPODArray copy(index.cbegin(), index.cend()); + + HashMap dict_map; + HashMap add_keys_map; for (auto val : index) { - if (val < max_val) - hash_map.insert({val, hash_map.size()}); + if (val < dict_size) + dict_map.insert({val, dict_map.size()}); + else + add_keys_map.insert({val, add_keys_map.size()}); } - auto index_map_col = ColumnVector::create(); - auto & index_data = index_map_col->getData(); + auto dictionary_map = ColumnVector::create(dict_map.size()); + auto additional_keys_map = ColumnVector::create(add_keys_map.size()); + auto & dict_data = dictionary_map->getData(); + auto & add_keys_data = additional_keys_map->getData(); - index_data.resize(hash_map.size()); - for (auto val : hash_map) - index_data[val.second] = val.first; + for (auto val : dict_map) + dict_data[val.second] = val.first; + + for (auto val : add_keys_map) + add_keys_data[val.second] = val.first - dict_size; for (auto & val : index) - val = val < max_val ? hash_map[val] - : val - max_val + hash_map.size(); + val = val < dict_size ? dict_map[val] + : add_keys_map[val] + dict_map.size(); - return index_map_col; + for (size_t i = 0; i < index.size(); ++i) + { + T expected = index[i] < dict_data.size() ? dict_data[index[i]] + : add_keys_data[index[i] - dict_data.size()] + dict_size; + if (expected != copy[i]) + throw Exception("Expected " + toString(expected) + ", but got " + toString(copy[i]), ErrorCodes::LOGICAL_ERROR); + + } + + return {std::move(dictionary_map), std::move(additional_keys_map)}; } - MutableColumnPtr mapIndexWithOverflow(IColumn & column, size_t max_size) + /// Update column and return map with old indexes. + /// Let N is the number of distinct values which are less than max_size; + /// old_column - column before function call; + /// new_column - column after function call; + /// map - function result (map.size() is N): + /// * if old_column[i] < max_size, than + /// map[new_column[i]] = old_column[i] + /// * else + /// new_column[i] = old_column[i] - max_size + N + IndexMapsWithAdditionalKeys mapIndexWithAdditionalKeys(IColumn & column, size_t dict_size) { if (auto * data_uint8 = getIndexesData(column)) - return mapIndexWithOverflow(*data_uint8, max_size); + return mapIndexWithAdditionalKeys(*data_uint8, dict_size); else if (auto * data_uint16 = getIndexesData(column)) - return mapIndexWithOverflow(*data_uint16, max_size); + return mapIndexWithAdditionalKeys(*data_uint16, dict_size); else if (auto * data_uint32 = getIndexesData(column)) - return mapIndexWithOverflow(*data_uint32, max_size); + return mapIndexWithAdditionalKeys(*data_uint32, dict_size); else if (auto * data_uint64 = getIndexesData(column)) - return mapIndexWithOverflow(*data_uint64, max_size); + return mapIndexWithAdditionalKeys(*data_uint64, dict_size); else - throw Exception("Indexes column for makeIndexWithOverflow must be ColumnUInt, got" + column.getName(), + throw Exception("Indexes column for mapIndexWithAdditionalKeys must be UInt, got" + column.getName(), ErrorCodes::LOGICAL_ERROR); } } @@ -419,57 +408,65 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( auto & global_dictionary = state_with_dictionary->global_dictionary; KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value); - auto unique_state = global_dictionary->getSerializableState(); - bool was_global_dictionary_written = unique_state.limit >= settings.max_dictionary_size; - - const auto & indexes = column_with_dictionary.getIndexesPtr(); - const auto & keys = column_with_dictionary.getUnique()->getSerializableState().column; - size_t max_limit = column.size() - offset; limit = limit ? std::min(limit, max_limit) : max_limit; - /// Create pair (used_keys, sub_index) which is the dictionary for [offset, offset + limit) range. - MutableColumnPtr sub_index = (*indexes->cut(offset, limit)).mutate(); - auto unique_indexes = mapUniqueIndex(*sub_index); - /// unique_indexes->index(*sub_index) == indexes[offset:offset + limit] - MutableColumnPtr used_keys = (*keys->index(*unique_indexes, 0)).mutate(); + auto sub_column = column_with_dictionary.cutAndCompact(offset, limit); + ColumnPtr positions = sub_column->getIndexesPtr(); + ColumnPtr keys = sub_column->getDictionary().getNestedColumn(); if (settings.max_dictionary_size) { /// Insert used_keys into global dictionary and update sub_index. - auto indexes_with_overflow = global_dictionary->uniqueInsertRangeWithOverflow(*used_keys, 0, used_keys->size(), + auto indexes_with_overflow = global_dictionary->uniqueInsertRangeWithOverflow(*keys, 0, keys->size(), settings.max_dictionary_size); - sub_index = (*indexes_with_overflow.indexes->index(*sub_index, 0)).mutate(); - used_keys = std::move(indexes_with_overflow.overflowed_keys); + size_t max_size = settings.max_dictionary_size + indexes_with_overflow.overflowed_keys->size(); + ColumnWithDictionary::Index(indexes_with_overflow.indexes->getPtr()).check(max_size); + + if (global_dictionary->size() > settings.max_dictionary_size) + throw Exception("Got dictionary with size " + toString(global_dictionary->size()) + + " but max dictionary size is " + toString(settings.max_dictionary_size), + ErrorCodes::LOGICAL_ERROR); + + positions = indexes_with_overflow.indexes->index(*positions, 0); + keys = std::move(indexes_with_overflow.overflowed_keys); + + if (global_dictionary->size() < settings.max_dictionary_size && !keys->empty()) + throw Exception("Has additional keys, but dict size is " + toString(global_dictionary->size()) + + " which is less then max dictionary size (" + toString(settings.max_dictionary_size) + ")", + ErrorCodes::LOGICAL_ERROR); } - bool need_additional_keys = !used_keys->empty(); + if (auto nullable_keys = typeid_cast(keys.get())) + keys = nullable_keys->getNestedColumnPtr(); + + bool need_additional_keys = !keys->empty(); bool need_dictionary = settings.max_dictionary_size != 0; - bool need_write_dictionary = !was_global_dictionary_written && unique_state.limit >= settings.max_dictionary_size; + bool need_write_dictionary = !settings.use_single_dictionary_for_part + && global_dictionary->size() >= settings.max_dictionary_size; - IndexesSerializationType index_version(*indexes_type, need_additional_keys, need_dictionary); + IndexesSerializationType index_version(*positions, need_additional_keys, need_dictionary); index_version.serialize(*indexes_stream); - unique_state = global_dictionary->getSerializableState(); - if (need_write_dictionary) { - /// Write global dictionary if it wasn't written and has too many keys. - UInt64 num_keys = unique_state.limit; + const auto & nested_column = global_dictionary->getNestedNotNullableColumn(); + UInt64 num_keys = nested_column->size(); writeIntBinary(num_keys, *keys_stream); - removeNullable(dictionary_type)->serializeBinaryBulk(*unique_state.column, *keys_stream, unique_state.offset, num_keys); + removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *keys_stream, 0, num_keys); + state_with_dictionary->global_dictionary = createColumnUnique(*dictionary_type); } if (need_additional_keys) { - UInt64 num_keys = used_keys->size(); + UInt64 num_keys = keys->size(); writeIntBinary(num_keys, *indexes_stream); - removeNullable(dictionary_type)->serializeBinaryBulk(*used_keys, *indexes_stream, 0, num_keys); + removeNullable(dictionary_type)->serializeBinaryBulk(*keys, *indexes_stream, 0, num_keys); } - UInt64 num_rows = sub_index->size(); + UInt64 num_rows = positions->size(); writeIntBinary(num_rows, *indexes_stream); - indexes_type->serializeBinaryBulk(*sub_index, *indexes_stream, 0, num_rows); + index_version.getDataType()->serializeBinaryBulk(*positions, *indexes_stream, 0, num_rows); } void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( @@ -507,8 +504,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( auto global_dict_keys = keys_type->createColumn(); keys_type->deserializeBinaryBulk(*global_dict_keys, *keys_stream, num_keys, 0); - auto column_unique = createColumnUnique(*dictionary_type, *indexes_type); - column_unique->uniqueInsertRangeFrom(*global_dict_keys, 0, num_keys); + auto column_unique = createColumnUnique(*dictionary_type, std::move(global_dict_keys)); state_with_dictionary->global_dictionary = std::move(column_unique); }; @@ -517,61 +513,60 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( UInt64 num_keys; readIntBinary(num_keys, *indexes_stream); auto keys_type = removeNullable(dictionary_type); - state_with_dictionary->additional_keys = keys_type->createColumn(); - keys_type->deserializeBinaryBulk(*state_with_dictionary->additional_keys, *indexes_stream, num_keys, 0); + auto additional_keys = keys_type->createColumn(); + keys_type->deserializeBinaryBulk(*additional_keys, *indexes_stream, num_keys, 0); + state_with_dictionary->additional_keys = std::move(additional_keys); }; - auto readIndexes = [this, state_with_dictionary, indexes_stream, &column_with_dictionary](UInt64 num_rows, - bool need_dictionary) + auto readIndexes = [this, state_with_dictionary, indexes_stream, &column_with_dictionary](UInt64 num_rows) { + auto indexes_type = state_with_dictionary->index_type.getDataType(); MutableColumnPtr indexes_column = indexes_type->createColumn(); indexes_type->deserializeBinaryBulk(*indexes_column, *indexes_stream, num_rows, 0); auto & global_dictionary = state_with_dictionary->global_dictionary; const auto & additional_keys = state_with_dictionary->additional_keys; - auto * column_unique = column_with_dictionary.getUnique(); - bool has_additional_keys = state_with_dictionary->additional_keys != nullptr; + bool has_additional_keys = state_with_dictionary->index_type.has_additional_keys; bool column_is_empty = column_with_dictionary.empty(); - bool column_with_global_dictionary = column_unique == global_dictionary.get(); - if (!has_additional_keys && (column_is_empty || column_with_global_dictionary)) + if (!state_with_dictionary->index_type.need_global_dictionary) + { + column_with_dictionary.insertRangeFromDictionaryEncodedColumn(*additional_keys, *indexes_column); + } + else if (!has_additional_keys) { if (column_is_empty) - column_with_dictionary.setUnique(global_dictionary); + column_with_dictionary.setSharedDictionary(global_dictionary); - column_with_dictionary.getIndexes()->insertRangeFrom(*indexes_column, 0, num_rows); - } - else if (!need_dictionary) - { - auto indexes = column_unique->uniqueInsertRangeFrom(*additional_keys, 0, additional_keys->size()); - column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(*indexes_column, 0), 0, num_rows); + auto local_column = ColumnWithDictionary::create(global_dictionary, std::move(indexes_column)); + column_with_dictionary.insertRangeFrom(*local_column, 0, num_rows); } else { - if (column_with_global_dictionary) + auto maps = mapIndexWithAdditionalKeys(*indexes_column, global_dictionary->size()); + + ColumnWithDictionary::Index(maps.additional_keys_map->getPtr()).check(additional_keys->size()); + + ColumnWithDictionary::Index(indexes_column->getPtr()).check( + maps.dictionary_map->size() + maps.additional_keys_map->size()); + + auto used_keys = (*std::move(global_dictionary->getNestedColumn()->index(*maps.dictionary_map, 0))).mutate(); + + if (!maps.additional_keys_map->empty()) { - auto unique_indexes = mapUniqueIndex(*column_with_dictionary.getIndexes()); - auto sub_keys = column_with_dictionary.getUnique()->getNestedColumn()->index(*unique_indexes, 0); - auto new_unique = createColumnUnique(*dictionary_type, *indexes_type); - auto new_idx = new_unique->uniqueInsertRangeFrom(*sub_keys, 0, sub_keys->size()); - column_with_dictionary.setUnique(std::move(new_unique)); - column_with_dictionary.setIndexes((*(new_idx->index(*column_with_dictionary.getIndexes(), 0))).mutate()); - column_unique = column_with_dictionary.getUnique(); + auto used_add_keys = additional_keys->index(*maps.additional_keys_map, 0); + + if (dictionary_type->isNullable()) + { + ColumnPtr null_map = ColumnUInt8::create(used_add_keys->size(), 0); + used_add_keys = ColumnNullable::create(used_add_keys, null_map); + } + + used_keys->insertRangeFrom(*used_add_keys, 0, used_add_keys->size()); } - auto index_map = mapIndexWithOverflow(*indexes_column, global_dictionary->size()); - auto used_keys = global_dictionary->getNestedColumn()->index(*index_map, 0); - auto indexes = column_unique->uniqueInsertRangeFrom(*used_keys, 0, used_keys->size()); - - if (additional_keys) - { - size_t num_keys = additional_keys->size(); - auto additional_indexes = column_unique->uniqueInsertRangeFrom(*additional_keys, 0, num_keys); - indexes->insertRangeFrom(*additional_indexes, 0, num_keys); - } - - column_with_dictionary.getIndexes()->insertRangeFrom(*indexes->index(*indexes_column, 0), 0, num_rows); + column_with_dictionary.insertRangeFromDictionaryEncodedColumn(*used_keys, *indexes_column); } }; @@ -596,7 +591,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( } size_t num_rows_to_read = std::min(limit, state_with_dictionary->num_pending_rows); - readIndexes(num_rows_to_read, state_with_dictionary->index_type.need_global_dictionary); + readIndexes(num_rows_to_read); limit -= num_rows_to_read; state_with_dictionary->num_pending_rows -= num_rows_to_read; } @@ -617,8 +612,8 @@ void DataTypeWithDictionary::serializeImpl( DataTypeWithDictionary::SerealizeFunctionPtr func, Args & ... args) const { auto & column_with_dictionary = getColumnWithDictionary(column); - size_t unique_row_number = column_with_dictionary.getIndexes()->getUInt(row_num); - (dictionary_type.get()->*func)(*column_with_dictionary.getUnique()->getNestedColumn(), unique_row_number, ostr, std::forward(args)...); + size_t unique_row_number = column_with_dictionary.getIndexes().getUInt(row_num); + (dictionary_type.get()->*func)(*column_with_dictionary.getDictionary().getNestedColumn(), unique_row_number, ostr, std::forward(args)...); } template @@ -627,77 +622,56 @@ void DataTypeWithDictionary::deserializeImpl( DataTypeWithDictionary::DeserealizeFunctionPtr func, Args & ... args) const { auto & column_with_dictionary = getColumnWithDictionary(column); - auto temp_column = column_with_dictionary.getUnique()->cloneEmpty(); + auto temp_column = column_with_dictionary.getDictionary().cloneEmpty(); (dictionary_type.get()->*func)(*temp_column, istr, std::forward(args)...); column_with_dictionary.insertFromFullColumn(*temp_column, 0); } -template -MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDataType & keys_type) +namespace { - return ColumnUnique::create(keys_type); + template + struct CreateColumnVector + { + MutableColumnUniquePtr & column; + const IDataType & keys_type; + const Creator & creator; + + CreateColumnVector(MutableColumnUniquePtr & column, const IDataType & keys_type, const Creator & creator) + : column(column), keys_type(keys_type), creator(creator) + { + } + + template + void operator()() + { + if (typeid_cast *>(&keys_type)) + column = creator((ColumnVector *)(nullptr)); + } + }; } -template +template MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDataType & keys_type, - const IDataType & indexes_type) -{ - if (typeid_cast(&indexes_type)) - return createColumnUniqueImpl(keys_type); - if (typeid_cast(&indexes_type)) - return createColumnUniqueImpl(keys_type); - if (typeid_cast(&indexes_type)) - return createColumnUniqueImpl(keys_type); - if (typeid_cast(&indexes_type)) - return createColumnUniqueImpl(keys_type); - - throw Exception("The type of indexes must be unsigned integer, but got " + indexes_type.getName(), - ErrorCodes::LOGICAL_ERROR); -} - -struct CreateColumnVector -{ - MutableColumnUniquePtr & column; - const IDataType & keys_type; - const IDataType & indexes_type; - const IDataType * nested_type; - - CreateColumnVector(MutableColumnUniquePtr & column, const IDataType & keys_type, const IDataType & indexes_type) - : column(column), keys_type(keys_type), indexes_type(indexes_type), nested_type(&keys_type) - { - if (auto nullable_type = typeid_cast(&keys_type)) - nested_type = nullable_type->getNestedType().get(); - } - - template - void operator()() - { - if (typeid_cast *>(nested_type)) - column = DataTypeWithDictionary::createColumnUniqueImpl>(keys_type, indexes_type); - } -}; - -MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type, - const IDataType & indexes_type) + const Creator & creator) { auto * type = &keys_type; - if (type->isNullable()) - type = static_cast(keys_type).getNestedType().get(); + if (auto * nullable_type = typeid_cast(&keys_type)) + type = nullable_type->getNestedType().get(); if (type->isString()) - return createColumnUniqueImpl(keys_type, indexes_type); + return creator((ColumnString *)(nullptr)); if (type->isFixedString()) - return createColumnUniqueImpl(keys_type, indexes_type); + return creator((ColumnFixedString *)(nullptr)); if (typeid_cast(type)) - return createColumnUniqueImpl>(keys_type, indexes_type); + return creator((ColumnVector *)(nullptr)); if (typeid_cast(type)) - return createColumnUniqueImpl>(keys_type, indexes_type); + return creator((ColumnVector *)(nullptr)); if (type->isNumber()) { MutableColumnUniquePtr column; - TypeListNumbers::forEach(CreateColumnVector(column, keys_type, indexes_type)); + TypeListNumbers::forEach(CreateColumnVector(column, *type, creator)); if (!column) throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR); @@ -709,10 +683,31 @@ MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataTyp ErrorCodes::LOGICAL_ERROR); } + +MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type) +{ + auto creator = [&](auto x) + { + using ColumnType = typename std::remove_pointer::type; + return ColumnUnique::create(keys_type); + }; + return createColumnUniqueImpl(keys_type, creator); +} + +MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys) +{ + auto creator = [&](auto x) + { + using ColumnType = typename std::remove_pointer::type; + return ColumnUnique::create(std::move(keys), keys_type.isNullable()); + }; + return createColumnUniqueImpl(keys_type, creator); +} + MutableColumnPtr DataTypeWithDictionary::createColumn() const { - MutableColumnPtr indexes = indexes_type->createColumn(); - MutableColumnPtr dictionary = createColumnUnique(*dictionary_type, *indexes_type); + MutableColumnPtr indexes = DataTypeUInt8().createColumn(); + MutableColumnPtr dictionary = createColumnUnique(*dictionary_type); return ColumnWithDictionary::create(std::move(dictionary), std::move(indexes)); } @@ -722,20 +717,17 @@ bool DataTypeWithDictionary::equals(const IDataType & rhs) const return false; auto & rhs_with_dictionary = static_cast(rhs); - return dictionary_type->equals(*rhs_with_dictionary.dictionary_type) - && indexes_type->equals(*rhs_with_dictionary.indexes_type); + return dictionary_type->equals(*rhs_with_dictionary.dictionary_type); } - static DataTypePtr create(const ASTPtr & arguments) { - if (!arguments || arguments->children.size() != 2) - throw Exception("WithDictionary data type family must have two arguments - type of elements and type of indices" - , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (!arguments || arguments->children.size() != 1) + throw Exception("WithDictionary data type family must have single argument - type of elements", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return std::make_shared(DataTypeFactory::instance().get(arguments->children[0]), - DataTypeFactory::instance().get(arguments->children[1])); + return std::make_shared(DataTypeFactory::instance().get(arguments->children[0])); } void registerDataTypeWithDictionary(DataTypeFactory & factory) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h index fe6848d389e..4fa7bc80ea0 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeWithDictionary.h @@ -9,17 +9,15 @@ class DataTypeWithDictionary : public IDataType { private: DataTypePtr dictionary_type; - DataTypePtr indexes_type; public: - DataTypeWithDictionary(DataTypePtr dictionary_type_, DataTypePtr indexes_type_); + DataTypeWithDictionary(DataTypePtr dictionary_type_); const DataTypePtr & getDictionaryType() const { return dictionary_type; } - const DataTypePtr & getIndexesType() const { return indexes_type; } String getName() const override { - return "WithDictionary(" + dictionary_type->getName() + ", " + indexes_type->getName() + ")"; + return "WithDictionary(" + dictionary_type->getName() + ")"; } const char * getFamilyName() const override { return "WithDictionary"; } @@ -146,7 +144,8 @@ public: bool onlyNull() const override { return false; } bool withDictionary() const override { return true; } - static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, const IDataType & indexes_type); + static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type); + static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys); private: @@ -164,14 +163,8 @@ private: void deserializeImpl(IColumn & column, ReadBuffer & istr, DeserealizeFunctionPtr func, Args & ... args) const; - template - static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type); - - template - static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const IDataType & indexes_type); - - - friend struct CreateColumnVector; + template + static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator); }; } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 5608f72f9e3..b180dd9365b 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -121,8 +121,10 @@ public: OutputStreamGetter getter; SubstreamPath path; - bool position_independent_encoding = true; size_t max_dictionary_size = 0; + bool use_single_dictionary_for_part = true; + + bool position_independent_encoding = true; }; struct DeserializeBinaryBulkSettings diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index b10fea32d8d..a6a2862ca8f 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1576,7 +1576,7 @@ private: if (from_with_dict) { auto * col_with_dict = typeid_cast(prev_arg_col.get()); - arg.column = col_with_dict->getUnique()->getNestedColumn(); + arg.column = col_with_dict->getDictionary().getNestedColumn(); arg.type = from_with_dict->getDictionaryType(); tmp_rows_count = arg.column->size(); @@ -1602,9 +1602,7 @@ private: if (from_with_dict) { auto res_keys = std::move(res.column); - - auto idx = col_with_dict->getUnique()->uniqueInsertRangeFrom(*res_keys, 0, res_keys->size()); - col_with_dict->getIndexes()->insertRangeFrom(*idx->index(*res_indexes, 0), 0, res_indexes->size()); + col_with_dict->insertRangeFromDictionaryEncodedColumn(*res_keys, *res_indexes); } else col_with_dict->insertRangeFromFullColumn(*res.column, 0, res.column->size()); diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index b2054190327..ebd7e3aa3db 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -1836,35 +1836,11 @@ public: }; -template -struct FunctionMakeDictionaryName; -template <> -struct FunctionMakeDictionaryName -{ - static constexpr auto name = "makeDictionaryUInt8"; -}; -template <> -struct FunctionMakeDictionaryName -{ - static constexpr auto name = "makeDictionaryUInt16"; -}; -template <> -struct FunctionMakeDictionaryName -{ - static constexpr auto name = "makeDictionaryUInt32"; -}; -template <> -struct FunctionMakeDictionaryName -{ - static constexpr auto name = "makeDictionaryUInt64"; -}; - -template class FunctionMakeDictionary: public IFunction { public: - static constexpr auto name = FunctionMakeDictionaryName::name; - static FunctionPtr create(const Context &) { return std::make_shared>(); } + static constexpr auto name = "makeDictionary"; + static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { return name; } @@ -1875,7 +1851,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - return std::make_shared(arguments[0], std::make_shared>()); + return std::make_shared(arguments[0]); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override @@ -1910,7 +1886,7 @@ public: throw Exception("First first argument of function dictionaryIndexes must be ColumnWithDictionary, but got" + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return type->getIndexesType(); + return std::make_shared(); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override @@ -1918,7 +1894,13 @@ public: auto arg_num = arguments[0]; const auto & arg = block.getByPosition(arg_num); auto & res = block.getByPosition(result); - res.column = typeid_cast(arg.column.get())->getIndexesPtr(); + auto indexes_col = typeid_cast(arg.column.get())->getIndexesPtr(); + auto new_indexes_col = ColumnUInt64::create(indexes_col->size()); + auto & data = new_indexes_col->getData(); + for (size_t i = 0; i < data.size(); ++i) + data[i] = indexes_col->getUInt(i); + + res.column = std::move(new_indexes_col); } }; @@ -1952,7 +1934,7 @@ public: const auto & arg = block.getByPosition(arg_num); auto & res = block.getByPosition(result); const auto * column_with_dictionary = typeid_cast(arg.column.get()); - res.column = column_with_dictionary->getUnique()->getNestedColumn()->cloneResized(arg.column->size()); + res.column = column_with_dictionary->getDictionary().getNestedColumn()->cloneResized(arg.column->size()); } }; @@ -2009,10 +1991,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); } diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 2a2492e4577..3d7d8ef4f73 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -254,7 +254,7 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar else { has_with_dictionary = true; - column_with_dict_size = column_with_dict->getUnique()->size(); + column_with_dict_size = column_with_dict->getDictionary().size(); indexes = column_with_dict->getIndexesPtr(); } } @@ -291,7 +291,7 @@ static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & ar + column.type->getName(), ErrorCodes::LOGICAL_ERROR); ColumnPtr new_column = convert_all_to_full ? column_with_dict->convertToFullColumn() - : column_with_dict->getUnique()->getNestedColumn(); + : column_with_dict->getDictionary().getNestedColumn(); temp_block.insert({new_column, type_with_dict->getDictionaryType(), column.name}); } @@ -355,7 +355,6 @@ void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) con struct ArgumentsWithoutDictionary { ColumnsWithTypeAndName arguments; - DataTypePtr common_index_type; bool all_without_dictionary = true; explicit ArgumentsWithoutDictionary(const ColumnsWithTypeAndName & args) @@ -372,12 +371,8 @@ struct ArgumentsWithoutDictionary arguments = args; } arguments[i].type = arg_with_dict->getDictionaryType(); - index_types.push_back(arg_with_dict->getIndexesType()); } } - - if (!all_without_dictionary) - common_index_type = getLeastSupertype(index_types); } }; @@ -387,13 +382,13 @@ DataTypePtr FunctionBuilderImpl::getReturnTypeWithoutDictionary(const ColumnsWit if (!arguments.empty() && useDefaultImplementationForNulls()) { - NullPresence null_presense = getNullPresense(arguments); + NullPresence null_presence = getNullPresense(arguments); - if (null_presense.has_null_constant) + if (null_presence.has_null_constant) { return makeNullable(std::make_shared()); } - if (null_presense.has_nullable) + if (null_presence.has_nullable) { Block nested_block = createBlockWithNestedColumns(Block(arguments), ext::collection_cast(ext::range(0, arguments.size()))); auto return_type = getReturnTypeImpl(ColumnsWithTypeAndName(nested_block.begin(), nested_block.end())); @@ -479,8 +474,7 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar ArgumentsWithoutDictionary arguments_without_dictionary(arguments); if (!arguments_without_dictionary.all_without_dictionary) return std::make_shared( - getReturnTypeWithoutDictionary(arguments_without_dictionary.arguments), - arguments_without_dictionary.common_index_type); + getReturnTypeWithoutDictionary(arguments_without_dictionary.arguments)); } return getReturnTypeWithoutDictionary(arguments); diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index c4d0d7654e6..aa2dbe0a579 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -264,6 +264,10 @@ struct Settings M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \ \ M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \ + \ + M(SettingUInt64, max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for WithDictionary type.") \ + M(SettingBool, use_single_dictionary_for_part, true, "WithDictionary type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.") \ + #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index c060f9e7982..d4947d9097a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -200,6 +200,11 @@ MergeTreeReader::Stream::Stream( getMark(right).offset_in_compressed_file - getMark(all_mark_ranges[i].begin).offset_in_compressed_file); } + /// Avoid empty buffer. May happen while reading dictionary for DataTypeWithDictionary. + /// For example: part has single dictionary and all marks point to the same position. + if (max_mark_range == 0) + max_mark_range = max_read_buffer_size; + size_t buffer_size = std::min(max_read_buffer_size, max_mark_range); /// Estimate size of the data to be read. @@ -329,6 +334,26 @@ void MergeTreeReader::Stream::seekToMark(size_t index) } +void MergeTreeReader::Stream::seekToStart() +{ + try + { + if (cached_buffer) + cached_buffer->seek(0, 0); + if (non_cached_buffer) + non_cached_buffer->seek(0, 0); + } + catch (Exception & e) + { + /// Better diagnostics. + if (e.code() == ErrorCodes::ARGUMENT_OUT_OF_BOUND) + e.addMessage("(while seeking to start of column " + path_prefix + ")"); + + throw; + } +} + + void MergeTreeReader::addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) { @@ -379,7 +404,12 @@ void MergeTreeReader::readData( Stream & stream = *it->second; - if (!continue_reading && !stream_for_prefix) + if (stream_for_prefix) + { + stream.seekToStart(); + continue_reading = false; + } + else if (!continue_reading) stream.seekToMark(from_mark); return stream.data_buffer; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 7271524ca86..a389918fdc4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -64,6 +64,7 @@ private: const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); void seekToMark(size_t index); + void seekToStart(); ReadBuffer * data_buffer; diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 0bf03531db5..a3da1e28319 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -97,9 +97,11 @@ void IMergedBlockOutputStream::writeData( bool skip_offsets, IDataType::SerializeBinaryBulkStatePtr & serialization_state) { - IDataType::SerializeBinaryBulkSettings settings; - settings.getter = createStreamGetter(name, offset_columns, skip_offsets); - settings.max_dictionary_size = 1024; + auto & settings = storage.context.getSettingsRef(); + IDataType::SerializeBinaryBulkSettings serialize_settings; + serialize_settings.getter = createStreamGetter(name, offset_columns, skip_offsets); + serialize_settings.max_dictionary_size = settings.max_dictionary_size; + serialize_settings.use_single_dictionary_for_part = settings.use_single_dictionary_for_part != 0; size_t size = column.size(); size_t prev_mark = 0; @@ -135,10 +137,10 @@ void IMergedBlockOutputStream::writeData( writeIntBinary(stream.plain_hashing.count(), stream.marks); writeIntBinary(stream.compressed.offset(), stream.marks); - }, settings.path); + }, serialize_settings.path); } - type.serializeBinaryBulkWithMultipleStreams(column, prev_mark, limit, settings, serialization_state); + type.serializeBinaryBulkWithMultipleStreams(column, prev_mark, limit, serialize_settings, serialization_state); /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path) @@ -154,7 +156,7 @@ void IMergedBlockOutputStream::writeData( return; column_streams[stream_name]->compressed.nextIfAtEnd(); - }, settings.path); + }, serialize_settings.path); prev_mark += limit; } @@ -168,7 +170,7 @@ void IMergedBlockOutputStream::writeData( String stream_name = IDataType::getFileNameForStream(name, substream_path); offset_columns.insert(stream_name); } - }, settings.path); + }, serialize_settings.path); } @@ -296,14 +298,16 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( MergeTreeData::DataPart::Checksums * additional_column_checksums) { /// Finish columns serialization. - IDataType::SerializeBinaryBulkSettings settings; - settings.max_dictionary_size = 1024; + auto & settings = storage.context.getSettingsRef(); + IDataType::SerializeBinaryBulkSettings serialize_settings; + serialize_settings.max_dictionary_size = settings.max_dictionary_size; + serialize_settings.use_single_dictionary_for_part = settings.use_single_dictionary_for_part != 0; OffsetColumns offset_columns; auto it = columns_list.begin(); for (size_t i = 0; i < columns_list.size(); ++i, ++it) { - settings.getter = createStreamGetter(it->name, offset_columns, false); - it->type->serializeBinaryBulkStateSuffix(settings, serialization_states[i]); + serialize_settings.getter = createStreamGetter(it->name, offset_columns, false); + it->type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]); } if (!total_column_list) @@ -550,14 +554,16 @@ void MergedColumnOnlyOutputStream::writeSuffix() MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums() { /// Finish columns serialization. - IDataType::SerializeBinaryBulkSettings settings; - settings.max_dictionary_size = 1024; + auto & settings = storage.context.getSettingsRef(); + IDataType::SerializeBinaryBulkSettings serialize_settings; + serialize_settings.max_dictionary_size = settings.max_dictionary_size; + serialize_settings.use_single_dictionary_for_part = settings.use_single_dictionary_for_part != 0; OffsetColumns offset_columns; for (size_t i = 0; i < header.columns(); ++i) { auto & column = header.safeGetByPosition(i); - settings.getter = createStreamGetter(column.name, offset_columns, skip_offsets); - column.type->serializeBinaryBulkStateSuffix(settings, serialization_states[i]); + serialize_settings.getter = createStreamGetter(column.name, offset_columns, skip_offsets); + column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]); } MergeTreeData::DataPart::Checksums checksums; From cbbbda25390fe807fca7f6e2ab8c3d9db3a4dc94 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 19 Jul 2018 17:37:22 +0300 Subject: [PATCH 110/315] Supported serialization of several dictionaries into single part. Run function on nested column of column with dictionary only if arguments have single column with dictionary and other columns are const. --- dbms/src/DataTypes/DataTypeWithDictionary.cpp | 45 +++-- dbms/src/Functions/IFunction.cpp | 186 +++++++++--------- 2 files changed, 120 insertions(+), 111 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeWithDictionary.cpp index 5259718c9b0..bdaf520db99 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeWithDictionary.cpp @@ -94,8 +94,9 @@ struct KeysSerializationVersion struct IndexesSerializationType { using SerializationType = UInt64; - static constexpr UInt64 NeedGlobalDictionaryBit = 1u << 8u; - static constexpr UInt64 HasAdditionalKeysBit = 1u << 9u; + static constexpr SerializationType NeedGlobalDictionaryBit = 1u << 8u; + static constexpr SerializationType HasAdditionalKeysBit = 1u << 9u; + static constexpr SerializationType NeedUpdateDictionary = 1u << 10u; enum Type { @@ -108,10 +109,11 @@ struct IndexesSerializationType Type type; bool has_additional_keys; bool need_global_dictionary; + bool need_update_dictionary; static constexpr SerializationType resetFlags(SerializationType type) { - return type & (~(HasAdditionalKeysBit | NeedGlobalDictionaryBit)); + return type & (~(HasAdditionalKeysBit | NeedGlobalDictionaryBit | NeedUpdateDictionary)); } static void checkType(SerializationType type) @@ -130,6 +132,8 @@ struct IndexesSerializationType val |= HasAdditionalKeysBit; if (need_global_dictionary) val |= NeedGlobalDictionaryBit; + if (need_update_dictionary) + val |= NeedUpdateDictionary; writeIntBinary(val, buffer); } @@ -140,11 +144,17 @@ struct IndexesSerializationType checkType(val); has_additional_keys = (val & HasAdditionalKeysBit) != 0; need_global_dictionary = (val & NeedGlobalDictionaryBit) != 0; + need_update_dictionary = (val & NeedUpdateDictionary) != 0; type = static_cast(resetFlags(val)); } - IndexesSerializationType(const IColumn & column, bool has_additional_keys, bool need_global_dictionary) - : has_additional_keys(has_additional_keys), need_global_dictionary(need_global_dictionary) + IndexesSerializationType(const IColumn & column, + bool has_additional_keys, + bool need_global_dictionary, + bool enumerate_dictionaries) + : has_additional_keys(has_additional_keys) + , need_global_dictionary(need_global_dictionary) + , need_update_dictionary(enumerate_dictionaries) { if (typeid_cast(&column)) type = TUInt8; @@ -181,11 +191,7 @@ struct SerializeStateWithDictionary : public IDataType::SerializeBinaryBulkState KeysSerializationVersion key_version; MutableColumnUniquePtr global_dictionary; - explicit SerializeStateWithDictionary( - UInt64 key_version, - MutableColumnUniquePtr && column_unique) - : key_version(key_version) - , global_dictionary(std::move(column_unique)) {} + explicit SerializeStateWithDictionary(UInt64 key_version) : key_version(key_version) {} }; struct DeserializeStateWithDictionary : public IDataType::DeserializeBinaryBulkState @@ -247,8 +253,7 @@ void DataTypeWithDictionary::serializeBinaryBulkStatePrefix( writeIntBinary(key_version, *stream); - auto column_unique = createColumnUnique(*dictionary_type); - state = std::make_shared(key_version, std::move(column_unique)); + state = std::make_shared(key_version); } void DataTypeWithDictionary::serializeBinaryBulkStateSuffix( @@ -273,6 +278,7 @@ void DataTypeWithDictionary::serializeBinaryBulkStateSuffix( UInt64 num_keys = nested_column->size(); writeIntBinary(num_keys, *stream); removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *stream, 0, num_keys); + state_with_dictionary->global_dictionary = nullptr; } } @@ -408,6 +414,10 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( auto & global_dictionary = state_with_dictionary->global_dictionary; KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value); + bool need_update_dictionary = global_dictionary == nullptr; + if (need_update_dictionary) + global_dictionary = createColumnUnique(*dictionary_type); + size_t max_limit = column.size() - offset; limit = limit ? std::min(limit, max_limit) : max_limit; @@ -445,7 +455,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( bool need_write_dictionary = !settings.use_single_dictionary_for_part && global_dictionary->size() >= settings.max_dictionary_size; - IndexesSerializationType index_version(*positions, need_additional_keys, need_dictionary); + IndexesSerializationType index_version(*positions, need_additional_keys, need_dictionary, need_update_dictionary); index_version.serialize(*indexes_stream); if (need_write_dictionary) @@ -454,7 +464,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( UInt64 num_keys = nested_column->size(); writeIntBinary(num_keys, *keys_stream); removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *keys_stream, 0, num_keys); - state_with_dictionary->global_dictionary = createColumnUnique(*dictionary_type); + state_with_dictionary->global_dictionary = nullptr; } if (need_additional_keys) @@ -577,9 +587,12 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( if (indexes_stream->eof()) break; - state_with_dictionary->index_type.deserialize(*indexes_stream); + auto & index_type = state_with_dictionary->index_type; + auto & global_dictionary = state_with_dictionary->global_dictionary; - if (state_with_dictionary->index_type.need_global_dictionary && !state_with_dictionary->global_dictionary) + index_type.deserialize(*indexes_stream); + + if (index_type.need_global_dictionary && (!global_dictionary || index_type.need_update_dictionary)) readDictionary(); if (state_with_dictionary->index_type.has_additional_keys) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 3d7d8ef4f73..126cb649c1e 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -238,105 +238,103 @@ void PreparedFunctionImpl::executeWithoutColumnsWithDictionary(Block & block, co executeImpl(block, args, result, input_rows_count); } -static Block removeColumnsWithDictionary(Block & block, const ColumnNumbers & args, size_t result, ColumnPtr & indexes) +static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(Block & block, const ColumnNumbers & args) { - bool has_with_dictionary = false; - bool convert_all_to_full = false; - size_t column_with_dict_size = 0; + size_t num_rows = 0; + ColumnPtr indexes; - for (auto & arg : args) + for (auto arg : args) { - const auto & column = block.getByPosition(arg).column; - if (auto * column_with_dict = checkAndGetColumn(column.get())) + ColumnWithTypeAndName & column = block.getByPosition(arg); + if (auto * column_with_dict = checkAndGetColumn(column.column.get())) { - if (has_with_dictionary) - convert_all_to_full = true; - else - { - has_with_dictionary = true; - column_with_dict_size = column_with_dict->getDictionary().size(); - indexes = column_with_dict->getIndexesPtr(); - } + if (indexes) + throw Exception("Expected single dictionary argument for function.", ErrorCodes::LOGICAL_ERROR); + + indexes = column_with_dict->getIndexesPtr(); + num_rows = column_with_dict->getDictionary().size(); } - else if (!checkColumn(column.get())) - convert_all_to_full = true; } - if (!has_with_dictionary || convert_all_to_full) - indexes = nullptr; + if (!indexes) + throw Exception("Expected column with dictionary for any function argument.", ErrorCodes::LOGICAL_ERROR); - if (!has_with_dictionary) - return {}; - - Block temp_block; - temp_block.insert(block.getByPosition(result)); + for (auto arg : args) { - auto & column = temp_block.getByPosition(0); - auto * type_with_dict = checkAndGetDataType(column.type.get()); - if (!type_with_dict) - throw Exception("Return type of function which has argument WithDictionary must be WithDictionary, got" - + column.type->getName(), ErrorCodes::LOGICAL_ERROR); + ColumnWithTypeAndName & column = block.getByPosition(arg); + if (auto * column_const = checkAndGetColumn(column.column.get())) + column.column = column_const->cloneResized(num_rows); + else if (auto * column_with_dict = checkAndGetColumn(column.column.get())) + { + auto * type_with_dict = checkAndGetDataType(column.type.get()); - column.type = type_with_dict->getDictionaryType(); + if (!type_with_dict) + throw Exception("Incompatible type for column with dictionary: " + column.type->getName(), + ErrorCodes::LOGICAL_ERROR); + + column.column = column_with_dict->getDictionary().getNestedColumn(); + column.type = type_with_dict->getDictionaryType(); + } } - for (auto & arg : args) + return indexes; +} + +static void convertColumnsWithDictionaryToFull(Block & block, const ColumnNumbers & args) +{ + for (auto arg : args) { - auto & column = block.getByPosition(arg); + ColumnWithTypeAndName & column = block.getByPosition(arg); if (auto * column_with_dict = checkAndGetColumn(column.column.get())) { auto * type_with_dict = checkAndGetDataType(column.type.get()); + if (!type_with_dict) - throw Exception("Column with dictionary must have type WithDictionary, but has" - + column.type->getName(), ErrorCodes::LOGICAL_ERROR); + throw Exception("Incompatible type for column with dictionary: " + column.type->getName(), + ErrorCodes::LOGICAL_ERROR); - ColumnPtr new_column = convert_all_to_full ? column_with_dict->convertToFullColumn() - : column_with_dict->getDictionary().getNestedColumn(); - - temp_block.insert({new_column, type_with_dict->getDictionaryType(), column.name}); + column.column = column_with_dict->convertToFullColumn(); + column.type = type_with_dict->getDictionaryType(); } - else if (auto * column_const = checkAndGetColumn(column.column.get())) - temp_block.insert({column_const->cloneResized(column_with_dict_size), column.type, column.name}); - else if (convert_all_to_full) - temp_block.insert(column); - else - throw Exception("Expected ColumnWithDictionary or ColumnConst, got" + column.column->getName(), - ErrorCodes::LOGICAL_ERROR); } - - return temp_block; } void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) { if (useDefaultImplementationForColumnsWithDictionary()) { - ColumnPtr indexes; - Block temp_block = removeColumnsWithDictionary(block, args, result, indexes); - if (temp_block) + auto & res = block.safeGetByPosition(result); + Block block_without_dicts = block.cloneEmpty(); + + for (auto arg : args) + block_without_dicts.safeGetByPosition(arg).column = block.safeGetByPosition(arg).column; + + if (res.type->withDictionary()) { - ColumnNumbers temp_numbers(args.size()); - for (size_t i = 0; i < args.size(); ++i) - temp_numbers[i] = i + 1; + ColumnPtr indexes = replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(block_without_dicts, args); - executeWithoutColumnsWithDictionary(temp_block, temp_numbers, 0, input_rows_count); - auto & temp_res_col = temp_block.getByPosition(0).column; - auto & res_col = block.getByPosition(result); - auto col_wit_dict_ptr = res_col.type->createColumn(); + executeWithoutColumnsWithDictionary(block_without_dicts, args, result, block_without_dicts.rows()); - auto * col_with_dict = typeid_cast(col_wit_dict_ptr.get()); - if (!col_with_dict) - throw Exception("Expected ColumnWithDictionary, got" + res_col.column->getName(), - ErrorCodes::LOGICAL_ERROR); + auto res_column = res.type->createColumn(); + auto * column_with_dictionary = typeid_cast(res_column.get()); - col_with_dict->insertRangeFromFullColumn(*temp_res_col, 0, temp_res_col->size()); - res_col.column = indexes ? col_with_dict->index(*indexes, 0) - : std::move(col_wit_dict_ptr); - return; + if (!column_with_dictionary) + throw Exception("Expected ColumnWithDictionary, got" + res_column->getName(), ErrorCodes::LOGICAL_ERROR); + + const auto & keys = block_without_dicts.safeGetByPosition(result).column; + column_with_dictionary->insertRangeFromDictionaryEncodedColumn(*keys, *indexes); + + res.column = std::move(res_column); + } + else + { + convertColumnsWithDictionaryToFull(block_without_dicts, args); + executeWithoutColumnsWithDictionary(block_without_dicts, args, result, block_without_dicts.rows()); + res.column = block_without_dicts.safeGetByPosition(result).column; } } - - executeWithoutColumnsWithDictionary(block, args, result, input_rows_count); + else + executeWithoutColumnsWithDictionary(block, args, result, input_rows_count); } void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const @@ -352,30 +350,6 @@ void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) con ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } -struct ArgumentsWithoutDictionary -{ - ColumnsWithTypeAndName arguments; - bool all_without_dictionary = true; - - explicit ArgumentsWithoutDictionary(const ColumnsWithTypeAndName & args) - { - DataTypes index_types; - for (size_t i = 0; i < args.size(); ++i) - { - const auto & arg = args[i]; - if (auto * arg_with_dict = typeid_cast(arg.type.get())) - { - if (all_without_dictionary) - { - all_without_dictionary = false; - arguments = args; - } - arguments[i].type = arg_with_dict->getDictionaryType(); - } - } - } -}; - DataTypePtr FunctionBuilderImpl::getReturnTypeWithoutDictionary(const ColumnsWithTypeAndName & arguments) const { checkNumberOfArguments(arguments.size()); @@ -471,10 +445,32 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar { if (useDefaultImplementationForColumnsWithDictionary()) { - ArgumentsWithoutDictionary arguments_without_dictionary(arguments); - if (!arguments_without_dictionary.all_without_dictionary) - return std::make_shared( - getReturnTypeWithoutDictionary(arguments_without_dictionary.arguments)); + bool has_type_with_dictionary = false; + bool can_run_function_on_dictionary = true; + + ColumnsWithTypeAndName args_without_dictionary(arguments); + + for (ColumnWithTypeAndName & arg : args_without_dictionary) + { + if (arg.column && arg.column->isColumnConst()) + continue; + + if (auto * type_with_dictionary = typeid_cast(arg.type.get())) + { + if (has_type_with_dictionary) + can_run_function_on_dictionary = false; + + has_type_with_dictionary = true; + arg.type = type_with_dictionary->getDictionaryType(); + } + else + can_run_function_on_dictionary = false; + } + + if (has_type_with_dictionary && can_run_function_on_dictionary) + return std::make_shared(getReturnTypeWithoutDictionary(args_without_dictionary)); + else + return getReturnTypeWithoutDictionary(args_without_dictionary); } return getReturnTypeWithoutDictionary(arguments); From e8bd73ff9b00c021cb212067e3ffce0e5cf4ac38 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 19 Jul 2018 21:20:52 +0300 Subject: [PATCH 111/315] Fixed function makeDictionary for column with dictionary argument. --- dbms/src/Functions/FunctionsMiscellaneous.cpp | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index ebd7e3aa3db..b352a196372 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -1848,9 +1848,13 @@ public: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { + if (arguments[0]->withDictionary()) + return arguments[0]; + return std::make_shared(arguments[0]); } @@ -1859,9 +1863,15 @@ public: auto arg_num = arguments[0]; const auto & arg = block.getByPosition(arg_num); auto & res = block.getByPosition(result); - auto column = res.type->createColumn(); - typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); - res.column = std::move(column); + + if (arg.type->withDictionary()) + res.column = arg.column; + else + { + auto column = res.type->createColumn(); + typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); + res.column = std::move(column); + } } }; From bf26fda38d3c1b2d82b9ca32262e53509c8c5114 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 19 Jul 2018 22:12:48 +0300 Subject: [PATCH 112/315] Added canBeExecutedOnDefaultArguments into IFunction. Don't run function on defaults is this flag is true. --- dbms/src/Columns/ColumnWithDictionary.cpp | 11 +++++++++++ dbms/src/Columns/ColumnWithDictionary.h | 9 ++++++++- dbms/src/Functions/FunctionsArithmetic.h | 12 +++++++----- dbms/src/Functions/IFunction.cpp | 15 ++++++++++++--- dbms/src/Functions/IFunction.h | 7 +++++++ 5 files changed, 45 insertions(+), 9 deletions(-) diff --git a/dbms/src/Columns/ColumnWithDictionary.cpp b/dbms/src/Columns/ColumnWithDictionary.cpp index 0f64d4bcbe0..00f38d085fa 100644 --- a/dbms/src/Columns/ColumnWithDictionary.cpp +++ b/dbms/src/Columns/ColumnWithDictionary.cpp @@ -300,6 +300,17 @@ void ColumnWithDictionary::compactIfSharedDictionary() } +ColumnWithDictionary::DictionaryEncodedColumn +ColumnWithDictionary::getMinimalDictionaryEncodedColumn(size_t offset, size_t limit) const +{ + MutableColumnPtr sub_indexes = (*std::move(idx.getPositions()->cut(offset, limit))).mutate(); + auto indexes_map = mapUniqueIndex(*sub_indexes); + auto sub_keys = getDictionary().getNestedColumn()->index(*indexes_map, 0); + + return {std::move(sub_keys), std::move(sub_indexes)}; +} + + ColumnWithDictionary::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {} ColumnWithDictionary::Index::Index(MutableColumnPtr && positions) : positions(std::move(positions)) diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnWithDictionary.h index aaced565e14..5d68dca5796 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnWithDictionary.h @@ -135,6 +135,7 @@ public: bool isFixedAndContiguous() const override { return getDictionary().isFixedAndContiguous(); } size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); } bool isNumeric() const override { return getDictionary().isNumeric(); } + bool withDictionary() const override { return true; } const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); } /// IColumnUnique & getUnique() { return static_cast(*column_unique->assumeMutable()); } @@ -155,7 +156,13 @@ public: /// Cut + compact. MutablePtr cutAndCompact(size_t start, size_t length) const; - bool withDictionary() const override { return true; } + struct DictionaryEncodedColumn + { + ColumnPtr dictionary; + ColumnPtr indexes; + }; + + DictionaryEncodedColumn getMinimalDictionaryEncodedColumn(size_t offset, size_t limit) const; class Index { diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index adbfdba175f..d6f4cd3a0f2 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -749,7 +749,7 @@ static bool castTypeToEither(const IDataType * type, F && f) } -template