From d7e25e143952707ad3121180c6ebf873ace83963 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Dec 2018 22:28:37 +0300 Subject: [PATCH 001/324] Always build ODBC bridge as a separate binary #3360 --- dbms/programs/CMakeLists.txt | 20 +++++++++---------- dbms/programs/main.cpp | 6 ------ dbms/programs/odbc-bridge/CMakeLists.txt | 10 ++++++---- dbms/src/Common/SharedLibrary.cpp | 6 +++--- dbms/src/Common/SharedLibrary.h | 5 +++-- .../Dictionaries/LibraryDictionarySource.cpp | 2 +- 6 files changed, 22 insertions(+), 27 deletions(-) diff --git a/dbms/programs/CMakeLists.txt b/dbms/programs/CMakeLists.txt index 9d7c6f2cda1..613b21cf48b 100644 --- a/dbms/programs/CMakeLists.txt +++ b/dbms/programs/CMakeLists.txt @@ -28,11 +28,18 @@ add_subdirectory (copier) add_subdirectory (format) add_subdirectory (clang) add_subdirectory (obfuscator) -add_subdirectory (odbc-bridge) + +if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) + add_subdirectory (odbc-bridge) +endif () if (CLICKHOUSE_SPLIT_BINARY) set (CLICKHOUSE_ALL_TARGETS clickhouse-server clickhouse-client clickhouse-local clickhouse-benchmark clickhouse-performance-test - clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-copier clickhouse-odbc-bridge) + clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-copier) + + if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) + list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge) + endif () if (USE_EMBEDDED_COMPILER) list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-clang clickhouse-lld) @@ -85,9 +92,6 @@ else () if (USE_EMBEDDED_COMPILER) target_link_libraries (clickhouse PRIVATE clickhouse-compiler-lib) endif () - if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) - target_link_libraries (clickhouse PRIVATE clickhouse-odbc-bridge-lib) - endif() set (CLICKHOUSE_BUNDLE) if (ENABLE_CLICKHOUSE_SERVER) @@ -140,12 +144,6 @@ else () install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-obfuscator DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE clickhouse-obfuscator) endif () - if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) - add_custom_target (clickhouse-odbc-bridge ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-odbc-bridge DEPENDS clickhouse) - install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-odbc-bridge DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-odbc-bridge) - endif () - # install always because depian package want this files: add_custom_target (clickhouse-clang ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-clang DEPENDS clickhouse) diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index 29d64213d9c..112803dab57 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -56,9 +56,6 @@ int mainEntryClickHouseClusterCopier(int argc, char ** argv); #if ENABLE_CLICKHOUSE_OBFUSCATOR int mainEntryClickHouseObfuscator(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_ODBC_BRIDGE || !defined(ENABLE_CLICKHOUSE_ODBC_BRIDGE) -int mainEntryClickHouseODBCBridge(int argc, char ** argv); -#endif #if USE_EMBEDDED_COMPILER @@ -105,9 +102,6 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_OBFUSCATOR {"obfuscator", mainEntryClickHouseObfuscator}, #endif -#if ENABLE_CLICKHOUSE_ODBC_BRIDGE || !defined(ENABLE_CLICKHOUSE_ODBC_BRIDGE) - {"odbc-bridge", mainEntryClickHouseODBCBridge}, -#endif #if USE_EMBEDDED_COMPILER {"clang", mainEntryClickHouseClang}, diff --git a/dbms/programs/odbc-bridge/CMakeLists.txt b/dbms/programs/odbc-bridge/CMakeLists.txt index a57c8c9c8cf..f7667aaea18 100644 --- a/dbms/programs/odbc-bridge/CMakeLists.txt +++ b/dbms/programs/odbc-bridge/CMakeLists.txt @@ -33,7 +33,9 @@ if (ENABLE_TESTS) add_subdirectory (tests) endif () -if (CLICKHOUSE_SPLIT_BINARY) - add_executable (clickhouse-odbc-bridge odbc-bridge.cpp) - target_link_libraries (clickhouse-odbc-bridge PRIVATE clickhouse-odbc-bridge-lib) -endif () +# clickhouse-odbc-bridge is always a separate binary. +# Reason: it must not export symbols from SSL, mariadb-client, etc. to not break ABI compatibility with ODBC drivers. + +add_executable (clickhouse-odbc-bridge odbc-bridge.cpp) +target_link_libraries (clickhouse-odbc-bridge PRIVATE clickhouse-odbc-bridge-lib) +install (TARGETS clickhouse-odbc-bridge RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) diff --git a/dbms/src/Common/SharedLibrary.cpp b/dbms/src/Common/SharedLibrary.cpp index 92083055098..30ed3bccaab 100644 --- a/dbms/src/Common/SharedLibrary.cpp +++ b/dbms/src/Common/SharedLibrary.cpp @@ -1,9 +1,9 @@ #include "SharedLibrary.h" #include -#include #include #include "Exception.h" + namespace DB { namespace ErrorCodes @@ -12,9 +12,9 @@ namespace ErrorCodes extern const int CANNOT_DLSYM; } -SharedLibrary::SharedLibrary(const std::string & path) +SharedLibrary::SharedLibrary(const std::string & path, int flags) { - handle = dlopen(path.c_str(), RTLD_LAZY); + handle = dlopen(path.c_str(), flags); if (!handle) throw Exception(std::string("Cannot dlopen: ") + dlerror(), ErrorCodes::CANNOT_DLOPEN); } diff --git a/dbms/src/Common/SharedLibrary.h b/dbms/src/Common/SharedLibrary.h index 96c8f6fe025..9d2b9bc7843 100644 --- a/dbms/src/Common/SharedLibrary.h +++ b/dbms/src/Common/SharedLibrary.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -8,12 +9,12 @@ namespace DB { - /** Allows you to open a dynamic library and get a pointer to a function from it. +/** Allows you to open a dynamic library and get a pointer to a function from it. */ class SharedLibrary : private boost::noncopyable { public: - explicit SharedLibrary(const std::string & path); + explicit SharedLibrary(const std::string & path, int flags = RTLD_LAZY); ~SharedLibrary(); diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index eec291321ad..fe6a294c1ac 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -135,7 +135,7 @@ LibraryDictionarySource::LibraryDictionarySource( "LibraryDictionarySource: Can't load lib " + toString() + ": " + Poco::File(path).path() + " - File doesn't exist", ErrorCodes::FILE_DOESNT_EXIST); description.init(sample_block); - library = std::make_shared(path); + library = std::make_shared(path, RTLD_LAZY | RTLD_DEEPBIND); settings = std::make_shared(getLibSettings(config, config_prefix + lib_config_settings)); if (auto libNew = library->tryGetstrings), decltype(&ClickHouseLibrary::log))>( "ClickHouseDictionary_v3_libNew")) From 9031be381ad9965dbe7ab1a528621b77d50e28e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Jan 2019 19:58:43 +0300 Subject: [PATCH 002/324] Attempt to implemnt global thread pool #4018 --- dbms/src/Databases/IDatabase.h | 4 +- .../src/Interpreters/InterpreterCreateQuery.h | 4 +- libs/libcommon/include/common/ThreadPool.h | 73 +++++++++++++++---- libs/libcommon/src/ThreadPool.cpp | 61 ++++++++++------ 4 files changed, 103 insertions(+), 39 deletions(-) diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 7b7b877b0e1..64a65c2fb5f 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -8,12 +8,10 @@ #include #include #include +#include #include -class ThreadPool; - - namespace DB { diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index e450ae0728e..40089e17d25 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -2,12 +2,12 @@ #include #include +#include -class ThreadPool; - namespace DB { + class Context; class ASTCreateQuery; class ASTExpressionList; diff --git a/libs/libcommon/include/common/ThreadPool.h b/libs/libcommon/include/common/ThreadPool.h index dd82c0c0399..02e1a02c58e 100644 --- a/libs/libcommon/include/common/ThreadPool.h +++ b/libs/libcommon/include/common/ThreadPool.h @@ -7,6 +7,7 @@ #include #include #include +#include /** Very simple thread pool similar to boost::threadpool. @@ -14,17 +15,22 @@ * - catches exceptions and rethrows on wait. */ -class ThreadPool +template +class ThreadPoolImpl { public: using Job = std::function; - /// Size is constant, all threads are created immediately. - explicit ThreadPool(size_t m_size); + /// Size is constant. Up to num_threads are created on demand and then run until shutdown. + explicit ThreadPoolImpl(size_t num_threads); - /// Add new job. Locks until free thread in pool become available or exception in one of threads was thrown. + /// queue_size - maximum number of running plus scheduled jobs. It can be greater than num_threads. Zero means unlimited. + ThreadPoolImpl(size_t num_threads, size_t queue_size); + + /// Add new job. Locks until number of active jobs is less than maximum or exception in one of threads was thrown. /// If an exception in some thread was thrown, method silently returns, and exception will be rethrown only on call to 'wait' function. - void schedule(Job job); + /// Priority: greater is higher. + void schedule(Job job, int priority = 0); /// Wait for all currently active jobs to be done. /// You may call schedule and wait many times in arbitary order. @@ -34,24 +40,40 @@ public: /// Waits for all threads. Doesn't rethrow exceptions (use 'wait' method to rethrow exceptions). /// You should not destroy object while calling schedule or wait methods from another threads. - ~ThreadPool(); + ~ThreadPoolImpl(); - size_t size() const { return m_size; } + size_t size() const { return num_threads; } - /// Returns number of active jobs. + /// Returns number of running and scheduled jobs. size_t active() const; private: mutable std::mutex mutex; - std::condition_variable has_free_thread; - std::condition_variable has_new_job_or_shutdown; + std::condition_variable job_finished; + std::condition_variable new_job_or_shutdown; + + const size_t num_threads; + const size_t queue_size; - const size_t m_size; size_t active_jobs = 0; bool shutdown = false; - std::queue jobs; - std::vector threads; + struct JobWithPriority + { + Job job; + int priority; + + JobWithPriority(Job job, int priority) + : job(job), priority(priority) {} + + bool operator< (const JobWithPriority & rhs) const + { + return priority < rhs.priority; + } + }; + + std::priority_queue jobs; + std::vector threads; std::exception_ptr first_exception; @@ -61,6 +83,31 @@ private: }; +using FreeThreadPool = ThreadPoolImpl; + +class GlobalThreadPool : public FreeThreadPool, public ext::singleton +{ +public: + GlobalThreadPool() : FreeThreadPool(10000) {} /// TODO: global blocking limit may lead to deadlocks. +}; + +class ThreadFromGlobalPool +{ +public: + ThreadFromGlobalPool(std::function func) + { + GlobalThreadPool::instance().schedule(func); + } + + void join() + { + /// noop, the std::thread will continue to run inside global pool. + } +}; + +using ThreadPool = ThreadPoolImpl; + + /// Allows to save first catched exception in jobs and postpone its rethrow. class ExceptionHandler { diff --git a/libs/libcommon/src/ThreadPool.cpp b/libs/libcommon/src/ThreadPool.cpp index e45e64853dc..e460acb3163 100644 --- a/libs/libcommon/src/ThreadPool.cpp +++ b/libs/libcommon/src/ThreadPool.cpp @@ -2,14 +2,21 @@ #include -ThreadPool::ThreadPool(size_t m_size) - : m_size(m_size) +template +ThreadPoolImpl::ThreadPoolImpl(size_t num_threads) + : ThreadPoolImpl(num_threads, num_threads) { - threads.reserve(m_size); +} + +template +ThreadPoolImpl::ThreadPoolImpl(size_t num_threads, size_t queue_size) + : num_threads(num_threads), queue_size(queue_size) +{ + threads.reserve(num_threads); try { - for (size_t i = 0; i < m_size; ++i) + for (size_t i = 0; i < num_threads; ++i) threads.emplace_back([this] { worker(); }); } catch (...) @@ -19,25 +26,30 @@ ThreadPool::ThreadPool(size_t m_size) } } -void ThreadPool::schedule(Job job) +template +void ThreadPoolImpl::schedule(Job job, int priority) { { std::unique_lock lock(mutex); - has_free_thread.wait(lock, [this] { return active_jobs < m_size || shutdown; }); + job_finished.wait(lock, [this] { return !queue_size || active_jobs < queue_size || shutdown; }); if (shutdown) return; - jobs.push(std::move(job)); + jobs.emplace(std::move(job), priority); ++active_jobs; + + if (threads.size() < std::min(num_threads, active_jobs)) + threads.emplace_back([this] { worker(); }); } - has_new_job_or_shutdown.notify_one(); + new_job_or_shutdown.notify_one(); } -void ThreadPool::wait() +template +void ThreadPoolImpl::wait() { { std::unique_lock lock(mutex); - has_free_thread.wait(lock, [this] { return active_jobs == 0; }); + job_finished.wait(lock, [this] { return active_jobs == 0; }); if (first_exception) { @@ -48,19 +60,21 @@ void ThreadPool::wait() } } -ThreadPool::~ThreadPool() +template +ThreadPoolImpl::~ThreadPoolImpl() { finalize(); } -void ThreadPool::finalize() +template +void ThreadPoolImpl::finalize() { { std::unique_lock lock(mutex); shutdown = true; } - has_new_job_or_shutdown.notify_all(); + new_job_or_shutdown.notify_all(); for (auto & thread : threads) thread.join(); @@ -68,14 +82,15 @@ void ThreadPool::finalize() threads.clear(); } -size_t ThreadPool::active() const +template +size_t ThreadPoolImpl::active() const { std::unique_lock lock(mutex); return active_jobs; } - -void ThreadPool::worker() +template +void ThreadPoolImpl::worker() { while (true) { @@ -84,12 +99,12 @@ void ThreadPool::worker() { std::unique_lock lock(mutex); - has_new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); }); + new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); }); need_shutdown = shutdown; if (!jobs.empty()) { - job = std::move(jobs.front()); + job = jobs.top().job; jobs.pop(); } else @@ -113,8 +128,8 @@ void ThreadPool::worker() shutdown = true; --active_jobs; } - has_free_thread.notify_all(); - has_new_job_or_shutdown.notify_all(); + job_finished.notify_all(); + new_job_or_shutdown.notify_all(); return; } } @@ -124,11 +139,15 @@ void ThreadPool::worker() --active_jobs; } - has_free_thread.notify_all(); + job_finished.notify_all(); } } +template class ThreadPoolImpl; +template class ThreadPoolImpl; + + void ExceptionHandler::setException(std::exception_ptr && exception) { std::unique_lock lock(mutex); From 79ac86375ea65b022872ebfdc73ebee374d6d730 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Jan 2019 20:07:04 +0300 Subject: [PATCH 003/324] Attempt to implemnt global thread pool #4018 --- contrib/jemalloc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc b/contrib/jemalloc index 41b7372eade..cd2931ad9bb 160000 --- a/contrib/jemalloc +++ b/contrib/jemalloc @@ -1 +1 @@ -Subproject commit 41b7372eadee941b9164751b8d4963f915d3ceae +Subproject commit cd2931ad9bbd78208565716ab102e86d858c2fff From 43774a38b8ffc1534a94c258f9b5bb679be2acbd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Jan 2019 20:20:06 +0300 Subject: [PATCH 004/324] Attempt to implemnt global thread pool #4018 --- libs/libcommon/src/ThreadPool.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/libs/libcommon/src/ThreadPool.cpp b/libs/libcommon/src/ThreadPool.cpp index e460acb3163..e3f03e18a46 100644 --- a/libs/libcommon/src/ThreadPool.cpp +++ b/libs/libcommon/src/ThreadPool.cpp @@ -13,17 +13,6 @@ ThreadPoolImpl::ThreadPoolImpl(size_t num_threads, size_t queue_size) : num_threads(num_threads), queue_size(queue_size) { threads.reserve(num_threads); - - try - { - for (size_t i = 0; i < num_threads; ++i) - threads.emplace_back([this] { worker(); }); - } - catch (...) - { - finalize(); - throw; - } } template From 13dd877026d7756f5ec1d31dc7e0860b1bb57a75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Jan 2019 21:38:11 +0300 Subject: [PATCH 005/324] Attempt to implemnt global thread pool #4018 --- libs/libcommon/include/common/ThreadPool.h | 32 ++++++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/libs/libcommon/include/common/ThreadPool.h b/libs/libcommon/include/common/ThreadPool.h index 02e1a02c58e..a8cf84dd7b5 100644 --- a/libs/libcommon/include/common/ThreadPool.h +++ b/libs/libcommon/include/common/ThreadPool.h @@ -94,15 +94,43 @@ public: class ThreadFromGlobalPool { public: + ThreadFromGlobalPool() {} + ThreadFromGlobalPool(std::function func) { - GlobalThreadPool::instance().schedule(func); + mutex = std::make_unique(); + /// The function object must be copyable, so we wrap lock_guard in shared_ptr. + GlobalThreadPool::instance().schedule([lock = std::make_shared>(*mutex), func = std::move(func)] { func(); }); + } + + ThreadFromGlobalPool(ThreadFromGlobalPool && rhs) + { + *this = std::move(rhs); + } + + ThreadFromGlobalPool & operator=(ThreadFromGlobalPool && rhs) + { + if (mutex) + std::terminate(); + mutex = std::move(rhs.mutex); + return *this; + } + + ~ThreadFromGlobalPool() + { + if (mutex) + std::terminate(); } void join() { - /// noop, the std::thread will continue to run inside global pool. + { + std::lock_guard lock(*mutex); + } + mutex.reset(); } +private: + std::unique_ptr mutex; /// Object must be moveable. }; using ThreadPool = ThreadPoolImpl; From abcd5a2a49a00364f1ec283abd066dfa93382046 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Jan 2019 22:12:36 +0300 Subject: [PATCH 006/324] Attempt to implemnt global thread pool #4018 --- dbms/programs/benchmark/Benchmark.cpp | 2 +- dbms/programs/copier/ClusterCopier.cpp | 2 +- .../performance-test/PerformanceTest.cpp | 2 +- dbms/src/Common/tests/CMakeLists.txt | 6 + dbms/src/Common/tests/gtest_rw_lock.cpp | 2 +- .../src/Common/tests/parallel_aggregation.cpp | 2 +- .../Common/tests/parallel_aggregation2.cpp | 2 +- .../Common/tests/thread_creation_latency.cpp | 2 +- dbms/src/Common/tests/thread_pool.cpp | 2 +- .../AsynchronousBlockInputStream.h | 2 +- ...ggregatedMemoryEfficientBlockInputStream.h | 2 +- .../PushingToViewsBlockOutputStream.cpp | 2 +- dbms/src/Databases/DatabaseFactory.h | 2 +- dbms/src/Databases/DatabaseOrdinary.cpp | 2 +- dbms/src/Databases/IDatabase.h | 2 +- dbms/src/IO/AsynchronousWriteBuffer.h | 2 +- dbms/src/Interpreters/Aggregator.h | 2 +- dbms/src/Interpreters/Compiler.h | 2 +- dbms/src/Interpreters/Context.h | 2 +- dbms/src/Interpreters/EmbeddedDictionaries.h | 2 +- .../src/Interpreters/InterpreterCreateQuery.h | 2 +- .../Interpreters/InterpreterSystemQuery.cpp | 2 +- dbms/src/Interpreters/loadMetadata.cpp | 2 +- .../src/Interpreters/tests/internal_iotop.cpp | 2 +- .../DistributedBlockOutputStream.h | 2 +- dbms/src/Storages/StorageDictionary.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 2 +- libs/libcommon/CMakeLists.txt | 3 - libs/libcommon/include/common/MultiVersion.h | 56 ------ libs/libcommon/include/common/ThreadPool.h | 151 ---------------- libs/libcommon/src/ThreadPool.cpp | 169 ------------------ libs/libcommon/src/tests/CMakeLists.txt | 2 - libs/libcommon/src/tests/multi_version.cpp | 56 ------ 33 files changed, 32 insertions(+), 463 deletions(-) delete mode 100644 libs/libcommon/include/common/MultiVersion.h delete mode 100644 libs/libcommon/include/common/ThreadPool.h delete mode 100644 libs/libcommon/src/ThreadPool.cpp delete mode 100644 libs/libcommon/src/tests/multi_version.cpp diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 9bd3bda825a..b366add0ba5 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 4f285c83f17..59635e8cd95 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -18,7 +18,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index 27bf986fc1b..ca5677e0dc5 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Common/tests/CMakeLists.txt b/dbms/src/Common/tests/CMakeLists.txt index ec9636ce664..163b52991ae 100644 --- a/dbms/src/Common/tests/CMakeLists.txt +++ b/dbms/src/Common/tests/CMakeLists.txt @@ -53,6 +53,12 @@ target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io) add_executable (thread_pool thread_pool.cpp) target_link_libraries (thread_pool PRIVATE clickhouse_common_io) +add_executable (thread_pool_2 thread_pool_2.cpp) +target_link_libraries (thread_pool_2 PRIVATE clickhouse_common_io) + +add_executable (multi_version multi_version.cpp) +target_link_libraries (multi_version PRIVATE clickhouse_common_io) + add_executable (array_cache array_cache.cpp) target_link_libraries (array_cache PRIVATE clickhouse_common_io) diff --git a/dbms/src/Common/tests/gtest_rw_lock.cpp b/dbms/src/Common/tests/gtest_rw_lock.cpp index 6fd16be64cd..c95be0d641d 100644 --- a/dbms/src/Common/tests/gtest_rw_lock.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Common/tests/parallel_aggregation.cpp b/dbms/src/Common/tests/parallel_aggregation.cpp index 15a193de3d7..ba430b0c58c 100644 --- a/dbms/src/Common/tests/parallel_aggregation.cpp +++ b/dbms/src/Common/tests/parallel_aggregation.cpp @@ -16,7 +16,7 @@ #include #include -#include +#include using Key = UInt64; diff --git a/dbms/src/Common/tests/parallel_aggregation2.cpp b/dbms/src/Common/tests/parallel_aggregation2.cpp index dc43442de08..699fb65e9dc 100644 --- a/dbms/src/Common/tests/parallel_aggregation2.cpp +++ b/dbms/src/Common/tests/parallel_aggregation2.cpp @@ -16,7 +16,7 @@ #include #include -#include +#include using Key = UInt64; diff --git a/dbms/src/Common/tests/thread_creation_latency.cpp b/dbms/src/Common/tests/thread_creation_latency.cpp index ef910a3e9f3..9b551f713c1 100644 --- a/dbms/src/Common/tests/thread_creation_latency.cpp +++ b/dbms/src/Common/tests/thread_creation_latency.cpp @@ -5,7 +5,7 @@ #include #include -#include +#include int x = 0; diff --git a/dbms/src/Common/tests/thread_pool.cpp b/dbms/src/Common/tests/thread_pool.cpp index 841cc740db1..23dba2aadec 100644 --- a/dbms/src/Common/tests/thread_pool.cpp +++ b/dbms/src/Common/tests/thread_pool.cpp @@ -1,4 +1,4 @@ -#include +#include /** Reproduces bug in ThreadPool. * It get stuck if we call 'wait' many times from many other threads simultaneously. diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index c790deb49c2..f5770411588 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index bdabd8cc1f8..612adcc3599 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index d3e7eaeb9a2..9651eb9e39f 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/dbms/src/Databases/DatabaseFactory.h b/dbms/src/Databases/DatabaseFactory.h index 00265a2454b..bb912ca377b 100644 --- a/dbms/src/Databases/DatabaseFactory.h +++ b/dbms/src/Databases/DatabaseFactory.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index cb1c7587080..3ca8cd17f71 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 64a65c2fb5f..90dfa229f11 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include diff --git a/dbms/src/IO/AsynchronousWriteBuffer.h b/dbms/src/IO/AsynchronousWriteBuffer.h index d54f18e9f5d..c87777450e7 100644 --- a/dbms/src/IO/AsynchronousWriteBuffer.h +++ b/dbms/src/IO/AsynchronousWriteBuffer.h @@ -4,7 +4,7 @@ #include -#include +#include #include diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index f51f620064f..bed147d627d 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/Compiler.h b/dbms/src/Interpreters/Compiler.h index 3f1fa8924ec..b79cf26e0f0 100644 --- a/dbms/src/Interpreters/Compiler.h +++ b/dbms/src/Interpreters/Compiler.h @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 72354753e36..ca90073436a 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/EmbeddedDictionaries.h b/dbms/src/Interpreters/EmbeddedDictionaries.h index a4f97308401..ad2dd404b3e 100644 --- a/dbms/src/Interpreters/EmbeddedDictionaries.h +++ b/dbms/src/Interpreters/EmbeddedDictionaries.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index 40089e17d25..0fc26847574 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index fc472ad8a9e..cae7e13e342 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 38e8407082d..0287990d4e5 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/tests/internal_iotop.cpp b/dbms/src/Interpreters/tests/internal_iotop.cpp index c1088819f85..3a171c0c845 100644 --- a/dbms/src/Interpreters/tests/internal_iotop.cpp +++ b/dbms/src/Interpreters/tests/internal_iotop.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h index 1c3dd8217e1..de802a09483 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/StorageDictionary.h b/dbms/src/Storages/StorageDictionary.h index cffaf8879cd..08a3f32093b 100644 --- a/dbms/src/Storages/StorageDictionary.h +++ b/dbms/src/Storages/StorageDictionary.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 5350b1db579..67774e7a113 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -44,7 +44,7 @@ #include -#include +#include #include #include diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 5c6c242407f..0c6455ece26 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -18,7 +18,6 @@ add_library (common ${LINK_MODE} src/mremap.cpp src/JSON.cpp src/getMemoryAmount.cpp - src/ThreadPool.cpp src/demangle.cpp src/SetTerminalEcho.cpp @@ -34,11 +33,9 @@ add_library (common ${LINK_MODE} include/common/mremap.h include/common/likely.h include/common/logger_useful.h - include/common/MultiVersion.h include/common/strong_typedef.h include/common/JSON.h include/common/getMemoryAmount.h - include/common/ThreadPool.h include/common/demangle.h include/common/SetTerminalEcho.h include/common/find_symbols.h diff --git a/libs/libcommon/include/common/MultiVersion.h b/libs/libcommon/include/common/MultiVersion.h deleted file mode 100644 index 3014689f861..00000000000 --- a/libs/libcommon/include/common/MultiVersion.h +++ /dev/null @@ -1,56 +0,0 @@ -#pragma once - -#include -#include - - -/** Allow to store and read-only usage of an object in several threads, - * and to atomically replace an object in another thread. - * The replacement is atomic and reading threads can work with different versions of an object. - * - * Usage: - * MultiVersion x; - * - on data update: - * x.set(new value); - * - on read-only usage: - * { - * MultiVersion::Version current_version = x.get(); - * // use *current_version - * } // now we finish own current version; if the version is outdated and no one else is using it - it will be destroyed. - * - * All methods are thread-safe. - */ -template -class MultiVersion -{ -public: - /// Version of object for usage. shared_ptr manage lifetime of version. - using Version = std::shared_ptr; - - /// Default initialization - by nullptr. - MultiVersion() = default; - - MultiVersion(std::unique_ptr && value) - { - set(std::move(value)); - } - - /// Obtain current version for read-only usage. Returns shared_ptr, that manages lifetime of version. - Version get() const - { - /// NOTE: is it possible to lock-free replace of shared_ptr? - std::lock_guard lock(mutex); - return current_version; - } - - /// Update an object with new version. - void set(std::unique_ptr && value) - { - std::lock_guard lock(mutex); - current_version = std::move(value); - } - -private: - Version current_version; - mutable std::mutex mutex; -}; diff --git a/libs/libcommon/include/common/ThreadPool.h b/libs/libcommon/include/common/ThreadPool.h deleted file mode 100644 index a8cf84dd7b5..00000000000 --- a/libs/libcommon/include/common/ThreadPool.h +++ /dev/null @@ -1,151 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include - - -/** Very simple thread pool similar to boost::threadpool. - * Advantages: - * - catches exceptions and rethrows on wait. - */ - -template -class ThreadPoolImpl -{ -public: - using Job = std::function; - - /// Size is constant. Up to num_threads are created on demand and then run until shutdown. - explicit ThreadPoolImpl(size_t num_threads); - - /// queue_size - maximum number of running plus scheduled jobs. It can be greater than num_threads. Zero means unlimited. - ThreadPoolImpl(size_t num_threads, size_t queue_size); - - /// Add new job. Locks until number of active jobs is less than maximum or exception in one of threads was thrown. - /// If an exception in some thread was thrown, method silently returns, and exception will be rethrown only on call to 'wait' function. - /// Priority: greater is higher. - void schedule(Job job, int priority = 0); - - /// Wait for all currently active jobs to be done. - /// You may call schedule and wait many times in arbitary order. - /// If any thread was throw an exception, first exception will be rethrown from this method, - /// and exception will be cleared. - void wait(); - - /// Waits for all threads. Doesn't rethrow exceptions (use 'wait' method to rethrow exceptions). - /// You should not destroy object while calling schedule or wait methods from another threads. - ~ThreadPoolImpl(); - - size_t size() const { return num_threads; } - - /// Returns number of running and scheduled jobs. - size_t active() const; - -private: - mutable std::mutex mutex; - std::condition_variable job_finished; - std::condition_variable new_job_or_shutdown; - - const size_t num_threads; - const size_t queue_size; - - size_t active_jobs = 0; - bool shutdown = false; - - struct JobWithPriority - { - Job job; - int priority; - - JobWithPriority(Job job, int priority) - : job(job), priority(priority) {} - - bool operator< (const JobWithPriority & rhs) const - { - return priority < rhs.priority; - } - }; - - std::priority_queue jobs; - std::vector threads; - std::exception_ptr first_exception; - - - void worker(); - - void finalize(); -}; - - -using FreeThreadPool = ThreadPoolImpl; - -class GlobalThreadPool : public FreeThreadPool, public ext::singleton -{ -public: - GlobalThreadPool() : FreeThreadPool(10000) {} /// TODO: global blocking limit may lead to deadlocks. -}; - -class ThreadFromGlobalPool -{ -public: - ThreadFromGlobalPool() {} - - ThreadFromGlobalPool(std::function func) - { - mutex = std::make_unique(); - /// The function object must be copyable, so we wrap lock_guard in shared_ptr. - GlobalThreadPool::instance().schedule([lock = std::make_shared>(*mutex), func = std::move(func)] { func(); }); - } - - ThreadFromGlobalPool(ThreadFromGlobalPool && rhs) - { - *this = std::move(rhs); - } - - ThreadFromGlobalPool & operator=(ThreadFromGlobalPool && rhs) - { - if (mutex) - std::terminate(); - mutex = std::move(rhs.mutex); - return *this; - } - - ~ThreadFromGlobalPool() - { - if (mutex) - std::terminate(); - } - - void join() - { - { - std::lock_guard lock(*mutex); - } - mutex.reset(); - } -private: - std::unique_ptr mutex; /// Object must be moveable. -}; - -using ThreadPool = ThreadPoolImpl; - - -/// Allows to save first catched exception in jobs and postpone its rethrow. -class ExceptionHandler -{ -public: - void setException(std::exception_ptr && exception); - void throwIfException(); - -private: - std::exception_ptr first_exception; - std::mutex mutex; -}; - -ThreadPool::Job createExceptionHandledJob(ThreadPool::Job job, ExceptionHandler & handler); diff --git a/libs/libcommon/src/ThreadPool.cpp b/libs/libcommon/src/ThreadPool.cpp deleted file mode 100644 index e3f03e18a46..00000000000 --- a/libs/libcommon/src/ThreadPool.cpp +++ /dev/null @@ -1,169 +0,0 @@ -#include -#include - - -template -ThreadPoolImpl::ThreadPoolImpl(size_t num_threads) - : ThreadPoolImpl(num_threads, num_threads) -{ -} - -template -ThreadPoolImpl::ThreadPoolImpl(size_t num_threads, size_t queue_size) - : num_threads(num_threads), queue_size(queue_size) -{ - threads.reserve(num_threads); -} - -template -void ThreadPoolImpl::schedule(Job job, int priority) -{ - { - std::unique_lock lock(mutex); - job_finished.wait(lock, [this] { return !queue_size || active_jobs < queue_size || shutdown; }); - if (shutdown) - return; - - jobs.emplace(std::move(job), priority); - ++active_jobs; - - if (threads.size() < std::min(num_threads, active_jobs)) - threads.emplace_back([this] { worker(); }); - } - new_job_or_shutdown.notify_one(); -} - -template -void ThreadPoolImpl::wait() -{ - { - std::unique_lock lock(mutex); - job_finished.wait(lock, [this] { return active_jobs == 0; }); - - if (first_exception) - { - std::exception_ptr exception; - std::swap(exception, first_exception); - std::rethrow_exception(exception); - } - } -} - -template -ThreadPoolImpl::~ThreadPoolImpl() -{ - finalize(); -} - -template -void ThreadPoolImpl::finalize() -{ - { - std::unique_lock lock(mutex); - shutdown = true; - } - - new_job_or_shutdown.notify_all(); - - for (auto & thread : threads) - thread.join(); - - threads.clear(); -} - -template -size_t ThreadPoolImpl::active() const -{ - std::unique_lock lock(mutex); - return active_jobs; -} - -template -void ThreadPoolImpl::worker() -{ - while (true) - { - Job job; - bool need_shutdown = false; - - { - std::unique_lock lock(mutex); - new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); }); - need_shutdown = shutdown; - - if (!jobs.empty()) - { - job = jobs.top().job; - jobs.pop(); - } - else - { - return; - } - } - - if (!need_shutdown) - { - try - { - job(); - } - catch (...) - { - { - std::unique_lock lock(mutex); - if (!first_exception) - first_exception = std::current_exception(); - shutdown = true; - --active_jobs; - } - job_finished.notify_all(); - new_job_or_shutdown.notify_all(); - return; - } - } - - { - std::unique_lock lock(mutex); - --active_jobs; - } - - job_finished.notify_all(); - } -} - - -template class ThreadPoolImpl; -template class ThreadPoolImpl; - - -void ExceptionHandler::setException(std::exception_ptr && exception) -{ - std::unique_lock lock(mutex); - if (!first_exception) - first_exception = std::move(exception); -} - -void ExceptionHandler::throwIfException() -{ - std::unique_lock lock(mutex); - if (first_exception) - std::rethrow_exception(first_exception); -} - - -ThreadPool::Job createExceptionHandledJob(ThreadPool::Job job, ExceptionHandler & handler) -{ - return [job{std::move(job)}, &handler] () - { - try - { - job(); - } - catch (...) - { - handler.setException(std::current_exception()); - } - }; -} - diff --git a/libs/libcommon/src/tests/CMakeLists.txt b/libs/libcommon/src/tests/CMakeLists.txt index 2d46cb7146d..355c6679362 100644 --- a/libs/libcommon/src/tests/CMakeLists.txt +++ b/libs/libcommon/src/tests/CMakeLists.txt @@ -5,7 +5,6 @@ add_executable (date_lut2 date_lut2.cpp) add_executable (date_lut3 date_lut3.cpp) add_executable (date_lut4 date_lut4.cpp) add_executable (date_lut_default_timezone date_lut_default_timezone.cpp) -add_executable (multi_version multi_version.cpp) add_executable (local_date_time_comparison local_date_time_comparison.cpp) add_executable (realloc-perf allocator.cpp) @@ -16,7 +15,6 @@ target_link_libraries (date_lut2 common ${PLATFORM_LIBS}) target_link_libraries (date_lut3 common ${PLATFORM_LIBS}) target_link_libraries (date_lut4 common ${PLATFORM_LIBS}) target_link_libraries (date_lut_default_timezone common ${PLATFORM_LIBS}) -target_link_libraries (multi_version common) target_link_libraries (local_date_time_comparison common) target_link_libraries (realloc-perf common) add_check(multi_version) diff --git a/libs/libcommon/src/tests/multi_version.cpp b/libs/libcommon/src/tests/multi_version.cpp deleted file mode 100644 index 9cab2e095e4..00000000000 --- a/libs/libcommon/src/tests/multi_version.cpp +++ /dev/null @@ -1,56 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -using T = std::string; -using MV = MultiVersion; -using Results = std::vector; - - -void thread1(MV & x, T & result) -{ - MV::Version v = x.get(); - result = *v; -} - -void thread2(MV & x, const char * result) -{ - x.set(std::make_unique(result)); -} - - -int main(int argc, char ** argv) -{ - try - { - const char * s1 = "Hello!"; - const char * s2 = "Goodbye!"; - - size_t n = 1000; - MV x(std::make_unique(s1)); - Results results(n); - - ThreadPool tp(8); - for (size_t i = 0; i < n; ++i) - { - tp.schedule(std::bind(thread1, std::ref(x), std::ref(results[i]))); - tp.schedule(std::bind(thread2, std::ref(x), (rand() % 2) ? s1 : s2)); - } - tp.wait(); - - for (size_t i = 0; i < n; ++i) - std::cerr << results[i] << " "; - std::cerr << std::endl; - } - catch (const Poco::Exception & e) - { - std::cerr << e.message() << std::endl; - throw; - } - - return 0; -} From 0afee1df23f631247803963779e2ee9b31ec0189 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Jan 2019 22:27:24 +0300 Subject: [PATCH 007/324] Attempt to implemnt global thread pool #4018 --- dbms/src/Common/tests/thread_pool_2.cpp | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 dbms/src/Common/tests/thread_pool_2.cpp diff --git a/dbms/src/Common/tests/thread_pool_2.cpp b/dbms/src/Common/tests/thread_pool_2.cpp new file mode 100644 index 00000000000..64a04c471f6 --- /dev/null +++ b/dbms/src/Common/tests/thread_pool_2.cpp @@ -0,0 +1,21 @@ +#include +#include +#include + + +int main(int, char **) +{ + std::atomic res{0}; + + for (size_t i = 0; i < 1000; ++i) + { + size_t threads = 16; + ThreadPool pool(threads); + for (size_t j = 0; j < threads; ++j) + pool.schedule([&]{ ++res; }); + pool.wait(); + } + + std::cerr << res << "\n"; + return 0; +} From a6bf5a7e28a8e49dc386ca48317bbab2839cc444 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Jan 2019 20:49:21 +0300 Subject: [PATCH 008/324] Addition to prev. revision --- dbms/src/Common/MultiVersion.h | 56 ++++++++ dbms/src/Common/ThreadPool.cpp | 168 ++++++++++++++++++++++++ dbms/src/Common/ThreadPool.h | 123 +++++++++++++++++ dbms/src/Common/tests/CMakeLists.txt | 1 + dbms/src/Common/tests/multi_version.cpp | 56 ++++++++ libs/libcommon/src/tests/CMakeLists.txt | 1 - 6 files changed, 404 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Common/MultiVersion.h create mode 100644 dbms/src/Common/ThreadPool.cpp create mode 100644 dbms/src/Common/ThreadPool.h create mode 100644 dbms/src/Common/tests/multi_version.cpp diff --git a/dbms/src/Common/MultiVersion.h b/dbms/src/Common/MultiVersion.h new file mode 100644 index 00000000000..3014689f861 --- /dev/null +++ b/dbms/src/Common/MultiVersion.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include + + +/** Allow to store and read-only usage of an object in several threads, + * and to atomically replace an object in another thread. + * The replacement is atomic and reading threads can work with different versions of an object. + * + * Usage: + * MultiVersion x; + * - on data update: + * x.set(new value); + * - on read-only usage: + * { + * MultiVersion::Version current_version = x.get(); + * // use *current_version + * } // now we finish own current version; if the version is outdated and no one else is using it - it will be destroyed. + * + * All methods are thread-safe. + */ +template +class MultiVersion +{ +public: + /// Version of object for usage. shared_ptr manage lifetime of version. + using Version = std::shared_ptr; + + /// Default initialization - by nullptr. + MultiVersion() = default; + + MultiVersion(std::unique_ptr && value) + { + set(std::move(value)); + } + + /// Obtain current version for read-only usage. Returns shared_ptr, that manages lifetime of version. + Version get() const + { + /// NOTE: is it possible to lock-free replace of shared_ptr? + std::lock_guard lock(mutex); + return current_version; + } + + /// Update an object with new version. + void set(std::unique_ptr && value) + { + std::lock_guard lock(mutex); + current_version = std::move(value); + } + +private: + Version current_version; + mutable std::mutex mutex; +}; diff --git a/dbms/src/Common/ThreadPool.cpp b/dbms/src/Common/ThreadPool.cpp new file mode 100644 index 00000000000..698c442fc9d --- /dev/null +++ b/dbms/src/Common/ThreadPool.cpp @@ -0,0 +1,168 @@ +#include + + +template +ThreadPoolImpl::ThreadPoolImpl(size_t num_threads) + : ThreadPoolImpl(num_threads, num_threads) +{ +} + +template +ThreadPoolImpl::ThreadPoolImpl(size_t num_threads, size_t queue_size) + : num_threads(num_threads), queue_size(queue_size) +{ + threads.reserve(num_threads); +} + +template +void ThreadPoolImpl::schedule(Job job, int priority) +{ + { + std::unique_lock lock(mutex); + job_finished.wait(lock, [this] { return !queue_size || active_jobs < queue_size || shutdown; }); + if (shutdown) + return; + + jobs.emplace(std::move(job), priority); + ++active_jobs; + + if (threads.size() < std::min(num_threads, active_jobs)) + threads.emplace_back([this] { worker(); }); + } + new_job_or_shutdown.notify_one(); +} + +template +void ThreadPoolImpl::wait() +{ + { + std::unique_lock lock(mutex); + job_finished.wait(lock, [this] { return active_jobs == 0; }); + + if (first_exception) + { + std::exception_ptr exception; + std::swap(exception, first_exception); + std::rethrow_exception(exception); + } + } +} + +template +ThreadPoolImpl::~ThreadPoolImpl() +{ + finalize(); +} + +template +void ThreadPoolImpl::finalize() +{ + { + std::unique_lock lock(mutex); + shutdown = true; + } + + new_job_or_shutdown.notify_all(); + + for (auto & thread : threads) + thread.join(); + + threads.clear(); +} + +template +size_t ThreadPoolImpl::active() const +{ + std::unique_lock lock(mutex); + return active_jobs; +} + +template +void ThreadPoolImpl::worker() +{ + while (true) + { + Job job; + bool need_shutdown = false; + + { + std::unique_lock lock(mutex); + new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); }); + need_shutdown = shutdown; + + if (!jobs.empty()) + { + job = jobs.top().job; + jobs.pop(); + } + else + { + return; + } + } + + if (!need_shutdown) + { + try + { + job(); + } + catch (...) + { + { + std::unique_lock lock(mutex); + if (!first_exception) + first_exception = std::current_exception(); + shutdown = true; + --active_jobs; + } + job_finished.notify_all(); + new_job_or_shutdown.notify_all(); + return; + } + } + + { + std::unique_lock lock(mutex); + --active_jobs; + } + + job_finished.notify_all(); + } +} + + +template class ThreadPoolImpl; +template class ThreadPoolImpl; + + +void ExceptionHandler::setException(std::exception_ptr && exception) +{ + std::unique_lock lock(mutex); + if (!first_exception) + first_exception = std::move(exception); +} + +void ExceptionHandler::throwIfException() +{ + std::unique_lock lock(mutex); + if (first_exception) + std::rethrow_exception(first_exception); +} + + +ThreadPool::Job createExceptionHandledJob(ThreadPool::Job job, ExceptionHandler & handler) +{ + return [job{std::move(job)}, &handler] () + { + try + { + job(); + } + catch (...) + { + handler.setException(std::current_exception()); + } + }; +} + diff --git a/dbms/src/Common/ThreadPool.h b/dbms/src/Common/ThreadPool.h new file mode 100644 index 00000000000..02e1a02c58e --- /dev/null +++ b/dbms/src/Common/ThreadPool.h @@ -0,0 +1,123 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + + +/** Very simple thread pool similar to boost::threadpool. + * Advantages: + * - catches exceptions and rethrows on wait. + */ + +template +class ThreadPoolImpl +{ +public: + using Job = std::function; + + /// Size is constant. Up to num_threads are created on demand and then run until shutdown. + explicit ThreadPoolImpl(size_t num_threads); + + /// queue_size - maximum number of running plus scheduled jobs. It can be greater than num_threads. Zero means unlimited. + ThreadPoolImpl(size_t num_threads, size_t queue_size); + + /// Add new job. Locks until number of active jobs is less than maximum or exception in one of threads was thrown. + /// If an exception in some thread was thrown, method silently returns, and exception will be rethrown only on call to 'wait' function. + /// Priority: greater is higher. + void schedule(Job job, int priority = 0); + + /// Wait for all currently active jobs to be done. + /// You may call schedule and wait many times in arbitary order. + /// If any thread was throw an exception, first exception will be rethrown from this method, + /// and exception will be cleared. + void wait(); + + /// Waits for all threads. Doesn't rethrow exceptions (use 'wait' method to rethrow exceptions). + /// You should not destroy object while calling schedule or wait methods from another threads. + ~ThreadPoolImpl(); + + size_t size() const { return num_threads; } + + /// Returns number of running and scheduled jobs. + size_t active() const; + +private: + mutable std::mutex mutex; + std::condition_variable job_finished; + std::condition_variable new_job_or_shutdown; + + const size_t num_threads; + const size_t queue_size; + + size_t active_jobs = 0; + bool shutdown = false; + + struct JobWithPriority + { + Job job; + int priority; + + JobWithPriority(Job job, int priority) + : job(job), priority(priority) {} + + bool operator< (const JobWithPriority & rhs) const + { + return priority < rhs.priority; + } + }; + + std::priority_queue jobs; + std::vector threads; + std::exception_ptr first_exception; + + + void worker(); + + void finalize(); +}; + + +using FreeThreadPool = ThreadPoolImpl; + +class GlobalThreadPool : public FreeThreadPool, public ext::singleton +{ +public: + GlobalThreadPool() : FreeThreadPool(10000) {} /// TODO: global blocking limit may lead to deadlocks. +}; + +class ThreadFromGlobalPool +{ +public: + ThreadFromGlobalPool(std::function func) + { + GlobalThreadPool::instance().schedule(func); + } + + void join() + { + /// noop, the std::thread will continue to run inside global pool. + } +}; + +using ThreadPool = ThreadPoolImpl; + + +/// Allows to save first catched exception in jobs and postpone its rethrow. +class ExceptionHandler +{ +public: + void setException(std::exception_ptr && exception); + void throwIfException(); + +private: + std::exception_ptr first_exception; + std::mutex mutex; +}; + +ThreadPool::Job createExceptionHandledJob(ThreadPool::Job job, ExceptionHandler & handler); diff --git a/dbms/src/Common/tests/CMakeLists.txt b/dbms/src/Common/tests/CMakeLists.txt index 163b52991ae..e4a72bbdeac 100644 --- a/dbms/src/Common/tests/CMakeLists.txt +++ b/dbms/src/Common/tests/CMakeLists.txt @@ -58,6 +58,7 @@ target_link_libraries (thread_pool_2 PRIVATE clickhouse_common_io) add_executable (multi_version multi_version.cpp) target_link_libraries (multi_version PRIVATE clickhouse_common_io) +add_check(multi_version) add_executable (array_cache array_cache.cpp) target_link_libraries (array_cache PRIVATE clickhouse_common_io) diff --git a/dbms/src/Common/tests/multi_version.cpp b/dbms/src/Common/tests/multi_version.cpp new file mode 100644 index 00000000000..0db6d74f4aa --- /dev/null +++ b/dbms/src/Common/tests/multi_version.cpp @@ -0,0 +1,56 @@ +#include +#include +#include +#include +#include +#include + + +using T = std::string; +using MV = MultiVersion; +using Results = std::vector; + + +void thread1(MV & x, T & result) +{ + MV::Version v = x.get(); + result = *v; +} + +void thread2(MV & x, const char * result) +{ + x.set(std::make_unique(result)); +} + + +int main(int argc, char ** argv) +{ + try + { + const char * s1 = "Hello!"; + const char * s2 = "Goodbye!"; + + size_t n = 1000; + MV x(std::make_unique(s1)); + Results results(n); + + ThreadPool tp(8); + for (size_t i = 0; i < n; ++i) + { + tp.schedule(std::bind(thread1, std::ref(x), std::ref(results[i]))); + tp.schedule(std::bind(thread2, std::ref(x), (rand() % 2) ? s1 : s2)); + } + tp.wait(); + + for (size_t i = 0; i < n; ++i) + std::cerr << results[i] << " "; + std::cerr << std::endl; + } + catch (const Poco::Exception & e) + { + std::cerr << e.message() << std::endl; + throw; + } + + return 0; +} diff --git a/libs/libcommon/src/tests/CMakeLists.txt b/libs/libcommon/src/tests/CMakeLists.txt index 355c6679362..35a7bddaa3a 100644 --- a/libs/libcommon/src/tests/CMakeLists.txt +++ b/libs/libcommon/src/tests/CMakeLists.txt @@ -17,7 +17,6 @@ target_link_libraries (date_lut4 common ${PLATFORM_LIBS}) target_link_libraries (date_lut_default_timezone common ${PLATFORM_LIBS}) target_link_libraries (local_date_time_comparison common) target_link_libraries (realloc-perf common) -add_check(multi_version) add_check(local_date_time_comparison) add_executable (unit_tests_libcommon gtest_json_test.cpp gtest_strong_typedef.cpp gtest_find_symbols.cpp) From 00a4b2cf8ac68cb0cc9e8e3ac1fc2826f0a05d7c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Jan 2019 21:51:57 +0300 Subject: [PATCH 009/324] Removing garbage, part 1 --- dbms/src/Common/CurrentThread.cpp | 64 +++++------------------ dbms/src/Common/CurrentThread.h | 21 +------- dbms/src/Common/ThreadStatus.cpp | 15 ++++-- dbms/src/Common/ThreadStatus.h | 22 ++++---- dbms/src/Common/tests/multi_version.cpp | 2 +- dbms/src/Common/tests/thread_pool_2.cpp | 2 +- dbms/src/Interpreters/ThreadStatusExt.cpp | 33 ++++++------ utils/iotest/iotest.cpp | 2 +- utils/iotest/iotest_aio.cpp | 9 ++-- utils/iotest/iotest_nonblock.cpp | 2 +- 10 files changed, 61 insertions(+), 111 deletions(-) diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index b2f165e5469..8c05c91bac3 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -2,6 +2,7 @@ #include "CurrentThread.h" #include +#include #include #include #include @@ -10,11 +11,6 @@ #include -#if defined(ARCADIA_ROOT) -# include -#endif - - namespace DB { @@ -23,91 +19,59 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -// Smoker's implementation to avoid thread_local usage: error: undefined symbol: __cxa_thread_atexit -#if defined(ARCADIA_ROOT) -struct ThreadStatusPtrHolder : ThreadStatusPtr -{ - ThreadStatusPtrHolder() { ThreadStatusPtr::operator=(ThreadStatus::create()); } -}; -struct ThreadScopePtrHolder : CurrentThread::ThreadScopePtr -{ - ThreadScopePtrHolder() { CurrentThread::ThreadScopePtr::operator=(std::make_shared()); } -}; -# define current_thread (*FastTlsSingleton()) -# define current_thread_scope (*FastTlsSingleton()) -#else -/// Order of current_thread and current_thread_scope matters -thread_local ThreadStatusPtr _current_thread = ThreadStatus::create(); -thread_local CurrentThread::ThreadScopePtr _current_thread_scope = std::make_shared(); -# define current_thread _current_thread -# define current_thread_scope _current_thread_scope -#endif - void CurrentThread::updatePerformanceCounters() { - get()->updatePerformanceCounters(); + get().updatePerformanceCounters(); } -ThreadStatusPtr CurrentThread::get() +ThreadStatus & CurrentThread::get() { -#ifndef NDEBUG - if (!current_thread || current_thread.use_count() <= 0) + if (unlikely(!current_thread)) 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; -} - -CurrentThread::ThreadScopePtr CurrentThread::getScope() -{ - return current_thread_scope; + return *current_thread; } ProfileEvents::Counters & CurrentThread::getProfileEvents() { - return current_thread->performance_counters; + return get().performance_counters; } MemoryTracker & CurrentThread::getMemoryTracker() { - return current_thread->memory_tracker; + return get().memory_tracker; } void CurrentThread::updateProgressIn(const Progress & value) { - current_thread->progress_in.incrementPiecewiseAtomically(value); + get().progress_in.incrementPiecewiseAtomically(value); } void CurrentThread::updateProgressOut(const Progress & value) { - current_thread->progress_out.incrementPiecewiseAtomically(value); + get().progress_out.incrementPiecewiseAtomically(value); } void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr & logs_queue) { - get()->attachInternalTextLogsQueue(logs_queue); + get().attachInternalTextLogsQueue(logs_queue); } 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 - - if (!current_thread || current_thread.use_count() <= 0) + if (!current_thread) return nullptr; - if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died) + if (get().getCurrentState() == ThreadStatus::ThreadState::Died) return nullptr; - return current_thread->getInternalTextLogsQueue(); + return get().getInternalTextLogsQueue(); } ThreadGroupStatusPtr CurrentThread::getGroup() { - return get()->getThreadGroup(); + return get().getThreadGroup(); } } diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h index 9820b3620ce..60e7993b5fc 100644 --- a/dbms/src/Common/CurrentThread.h +++ b/dbms/src/Common/CurrentThread.h @@ -32,7 +32,7 @@ class CurrentThread { public: /// Handler to current thread - static ThreadStatusPtr get(); + static ThreadStatus & get(); /// Group to which belongs current thread static ThreadGroupStatusPtr getGroup(); @@ -85,25 +85,6 @@ public: bool log_peak_memory_usage_in_destructor = true; }; - /// Implicitly finalizes current thread in the destructor - class ThreadScope - { - public: - void (*deleter)() = nullptr; - - ThreadScope() = default; - ~ThreadScope() - { - if (deleter) - deleter(); - - /// std::terminate on exception: this is Ok. - } - }; - - using ThreadScopePtr = std::shared_ptr; - static ThreadScopePtr getScope(); - private: static void defaultThreadDeleter(); }; diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 0ee09d527ce..3d66c33f032 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -21,10 +21,13 @@ namespace ErrorCodes } +thread_local ThreadStatusPtr current_thread = nullptr; + + TasksStatsCounters TasksStatsCounters::current() { TasksStatsCounters res; - CurrentThread::get()->taskstats_getter->getStat(res.stat, CurrentThread::get()->os_thread_id); + CurrentThread::get().taskstats_getter->getStat(res.stat, CurrentThread::get().os_thread_id); return res; } @@ -39,17 +42,19 @@ ThreadStatus::ThreadStatus() memory_tracker.setDescription("(for thread)"); log = &Poco::Logger::get("ThreadStatus"); + current_thread = this; + /// 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() +ThreadStatus::~ThreadStatus() { - return ThreadStatusPtr(new ThreadStatus); + if (deleter) + deleter(); + current_thread = nullptr; } -ThreadStatus::~ThreadStatus() = default; - void ThreadStatus::initPerformanceCounters() { performance_counters_finalized = false; diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 3f7a91a54f0..19c60f5cfc7 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -9,6 +9,8 @@ #include #include #include +#include +#include namespace Poco @@ -23,7 +25,7 @@ namespace DB class Context; class QueryStatus; class ThreadStatus; -using ThreadStatusPtr = std::shared_ptr; +using ThreadStatusPtr = ThreadStatus*; class QueryThreadLog; struct TasksStatsCounters; struct RUsageCounters; @@ -67,14 +69,20 @@ public: using ThreadGroupStatusPtr = std::shared_ptr; +extern thread_local ThreadStatusPtr current_thread; + /** Encapsulates all per-thread info (ProfileEvents, MemoryTracker, query_id, query context, etc.). - * Used inside thread-local variable. See variables in CurrentThread.cpp + * The object must be created in thread function and destroyed in the same thread before the exit. + * It is accessed through thread-local pointer. * * This object should be used only via "CurrentThread", see CurrentThread.h */ -class ThreadStatus : public std::enable_shared_from_this +class ThreadStatus : public boost::noncopyable { public: + ThreadStatus(); + ~ThreadStatus(); + /// 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) @@ -88,8 +96,8 @@ public: Progress progress_in; Progress progress_out; -public: - static ThreadStatusPtr create(); + using Deleter = std::function; + Deleter deleter; ThreadGroupStatusPtr getThreadGroup() const { @@ -136,11 +144,7 @@ public: /// 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(); - protected: - ThreadStatus(); - void initPerformanceCounters(); void logToQueryThreadLog(QueryThreadLog & thread_log); diff --git a/dbms/src/Common/tests/multi_version.cpp b/dbms/src/Common/tests/multi_version.cpp index 0db6d74f4aa..605cb3f0d62 100644 --- a/dbms/src/Common/tests/multi_version.cpp +++ b/dbms/src/Common/tests/multi_version.cpp @@ -23,7 +23,7 @@ void thread2(MV & x, const char * result) } -int main(int argc, char ** argv) +int main(int, char **) { try { diff --git a/dbms/src/Common/tests/thread_pool_2.cpp b/dbms/src/Common/tests/thread_pool_2.cpp index 64a04c471f6..029c3695e36 100644 --- a/dbms/src/Common/tests/thread_pool_2.cpp +++ b/dbms/src/Common/tests/thread_pool_2.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include int main(int, char **) diff --git a/dbms/src/Interpreters/ThreadStatusExt.cpp b/dbms/src/Interpreters/ThreadStatusExt.cpp index eac9251cdf0..20a58de98f8 100644 --- a/dbms/src/Interpreters/ThreadStatusExt.cpp +++ b/dbms/src/Interpreters/ThreadStatusExt.cpp @@ -36,7 +36,7 @@ String ThreadStatus::getQueryID() void CurrentThread::defaultThreadDeleter() { - ThreadStatus & thread = *CurrentThread::get(); + ThreadStatus & thread = CurrentThread::get(); LOG_TRACE(thread.log, "Thread " << thread.thread_number << " exited"); thread.detachQuery(true, true); } @@ -51,8 +51,8 @@ void ThreadStatus::initializeQuery() 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()); + thread_group->master_thread = this; + thread_group->thread_statuses.emplace(thread_number, this); initPerformanceCounters(); thread_state = ThreadState::AttachedToQuery; @@ -87,7 +87,7 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool if (!global_context) global_context = thread_group->global_context; - if (!thread_group->thread_statuses.emplace(thread_number, shared_from_this()).second) + if (!thread_group->thread_statuses.emplace(thread_number, this).second) throw Exception("Thread " + std::to_string(thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); } @@ -193,48 +193,47 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) void CurrentThread::initializeQuery() { - get()->initializeQuery(); - getScope()->deleter = CurrentThread::defaultThreadDeleter; + get().initializeQuery(); + get().deleter = CurrentThread::defaultThreadDeleter; } void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group) { - get()->attachQuery(thread_group, true); - getScope()->deleter = CurrentThread::defaultThreadDeleter; + get().attachQuery(thread_group, true); + get().deleter = CurrentThread::defaultThreadDeleter; } void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group) { - get()->attachQuery(thread_group, false); - getScope()->deleter = CurrentThread::defaultThreadDeleter; + get().attachQuery(thread_group, false); + get().deleter = CurrentThread::defaultThreadDeleter; } std::string CurrentThread::getCurrentQueryID() { - if (!get() || get().use_count() <= 0) + if (!current_thread) return {}; - - return get()->getQueryID(); + return get().getQueryID(); } void CurrentThread::attachQueryContext(Context & query_context) { - return get()->attachQueryContext(query_context); + return get().attachQueryContext(query_context); } void CurrentThread::finalizePerformanceCounters() { - get()->finalizePerformanceCounters(); + get().finalizePerformanceCounters(); } void CurrentThread::detachQuery() { - get()->detachQuery(false); + get().detachQuery(false); } void CurrentThread::detachQueryIfNotDetached() { - get()->detachQuery(true); + get().detachQuery(true); } diff --git a/utils/iotest/iotest.cpp b/utils/iotest/iotest.cpp index 3134a49056d..499eb9b464f 100644 --- a/utils/iotest/iotest.cpp +++ b/utils/iotest/iotest.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/utils/iotest/iotest_aio.cpp b/utils/iotest/iotest_aio.cpp index 1dcb8ea2ae7..77846e1ca80 100644 --- a/utils/iotest/iotest_aio.cpp +++ b/utils/iotest/iotest_aio.cpp @@ -1,5 +1,5 @@ #if __APPLE__ || __FreeBSD__ -int main(int argc, char ** argv) { return 0; } +int main(int, char **) { return 0; } #else #include @@ -11,7 +11,7 @@ int main(int argc, char ** argv) { return 0; } #include #include #include -#include +#include #include #include #include @@ -22,10 +22,7 @@ int main(int argc, char ** argv) { return 0; } #include #include #include - -#if !defined(__APPLE__) && !defined(__FreeBSD__) - #include -#endif +#include #include diff --git a/utils/iotest/iotest_nonblock.cpp b/utils/iotest/iotest_nonblock.cpp index f85e8df91f6..62871f6c162 100644 --- a/utils/iotest/iotest_nonblock.cpp +++ b/utils/iotest/iotest_nonblock.cpp @@ -16,7 +16,7 @@ #include #include #include -#include +#include #include #include From aec5570b6ecffdb701f17f7c107df28f8df88b4b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Jan 2019 13:59:58 +0300 Subject: [PATCH 010/324] Attempt to implemnt global thread pool #4018 --- dbms/src/Common/MultiVersion.h | 56 ++++++++ dbms/src/Common/ThreadPool.cpp | 169 ++++++++++++++++++++++++ dbms/src/Common/ThreadPool.h | 151 +++++++++++++++++++++ dbms/src/Common/tests/multi_version.cpp | 56 ++++++++ 4 files changed, 432 insertions(+) create mode 100644 dbms/src/Common/MultiVersion.h create mode 100644 dbms/src/Common/ThreadPool.cpp create mode 100644 dbms/src/Common/ThreadPool.h create mode 100644 dbms/src/Common/tests/multi_version.cpp diff --git a/dbms/src/Common/MultiVersion.h b/dbms/src/Common/MultiVersion.h new file mode 100644 index 00000000000..3014689f861 --- /dev/null +++ b/dbms/src/Common/MultiVersion.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include + + +/** Allow to store and read-only usage of an object in several threads, + * and to atomically replace an object in another thread. + * The replacement is atomic and reading threads can work with different versions of an object. + * + * Usage: + * MultiVersion x; + * - on data update: + * x.set(new value); + * - on read-only usage: + * { + * MultiVersion::Version current_version = x.get(); + * // use *current_version + * } // now we finish own current version; if the version is outdated and no one else is using it - it will be destroyed. + * + * All methods are thread-safe. + */ +template +class MultiVersion +{ +public: + /// Version of object for usage. shared_ptr manage lifetime of version. + using Version = std::shared_ptr; + + /// Default initialization - by nullptr. + MultiVersion() = default; + + MultiVersion(std::unique_ptr && value) + { + set(std::move(value)); + } + + /// Obtain current version for read-only usage. Returns shared_ptr, that manages lifetime of version. + Version get() const + { + /// NOTE: is it possible to lock-free replace of shared_ptr? + std::lock_guard lock(mutex); + return current_version; + } + + /// Update an object with new version. + void set(std::unique_ptr && value) + { + std::lock_guard lock(mutex); + current_version = std::move(value); + } + +private: + Version current_version; + mutable std::mutex mutex; +}; diff --git a/dbms/src/Common/ThreadPool.cpp b/dbms/src/Common/ThreadPool.cpp new file mode 100644 index 00000000000..e3f03e18a46 --- /dev/null +++ b/dbms/src/Common/ThreadPool.cpp @@ -0,0 +1,169 @@ +#include +#include + + +template +ThreadPoolImpl::ThreadPoolImpl(size_t num_threads) + : ThreadPoolImpl(num_threads, num_threads) +{ +} + +template +ThreadPoolImpl::ThreadPoolImpl(size_t num_threads, size_t queue_size) + : num_threads(num_threads), queue_size(queue_size) +{ + threads.reserve(num_threads); +} + +template +void ThreadPoolImpl::schedule(Job job, int priority) +{ + { + std::unique_lock lock(mutex); + job_finished.wait(lock, [this] { return !queue_size || active_jobs < queue_size || shutdown; }); + if (shutdown) + return; + + jobs.emplace(std::move(job), priority); + ++active_jobs; + + if (threads.size() < std::min(num_threads, active_jobs)) + threads.emplace_back([this] { worker(); }); + } + new_job_or_shutdown.notify_one(); +} + +template +void ThreadPoolImpl::wait() +{ + { + std::unique_lock lock(mutex); + job_finished.wait(lock, [this] { return active_jobs == 0; }); + + if (first_exception) + { + std::exception_ptr exception; + std::swap(exception, first_exception); + std::rethrow_exception(exception); + } + } +} + +template +ThreadPoolImpl::~ThreadPoolImpl() +{ + finalize(); +} + +template +void ThreadPoolImpl::finalize() +{ + { + std::unique_lock lock(mutex); + shutdown = true; + } + + new_job_or_shutdown.notify_all(); + + for (auto & thread : threads) + thread.join(); + + threads.clear(); +} + +template +size_t ThreadPoolImpl::active() const +{ + std::unique_lock lock(mutex); + return active_jobs; +} + +template +void ThreadPoolImpl::worker() +{ + while (true) + { + Job job; + bool need_shutdown = false; + + { + std::unique_lock lock(mutex); + new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); }); + need_shutdown = shutdown; + + if (!jobs.empty()) + { + job = jobs.top().job; + jobs.pop(); + } + else + { + return; + } + } + + if (!need_shutdown) + { + try + { + job(); + } + catch (...) + { + { + std::unique_lock lock(mutex); + if (!first_exception) + first_exception = std::current_exception(); + shutdown = true; + --active_jobs; + } + job_finished.notify_all(); + new_job_or_shutdown.notify_all(); + return; + } + } + + { + std::unique_lock lock(mutex); + --active_jobs; + } + + job_finished.notify_all(); + } +} + + +template class ThreadPoolImpl; +template class ThreadPoolImpl; + + +void ExceptionHandler::setException(std::exception_ptr && exception) +{ + std::unique_lock lock(mutex); + if (!first_exception) + first_exception = std::move(exception); +} + +void ExceptionHandler::throwIfException() +{ + std::unique_lock lock(mutex); + if (first_exception) + std::rethrow_exception(first_exception); +} + + +ThreadPool::Job createExceptionHandledJob(ThreadPool::Job job, ExceptionHandler & handler) +{ + return [job{std::move(job)}, &handler] () + { + try + { + job(); + } + catch (...) + { + handler.setException(std::current_exception()); + } + }; +} + diff --git a/dbms/src/Common/ThreadPool.h b/dbms/src/Common/ThreadPool.h new file mode 100644 index 00000000000..a8cf84dd7b5 --- /dev/null +++ b/dbms/src/Common/ThreadPool.h @@ -0,0 +1,151 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + + +/** Very simple thread pool similar to boost::threadpool. + * Advantages: + * - catches exceptions and rethrows on wait. + */ + +template +class ThreadPoolImpl +{ +public: + using Job = std::function; + + /// Size is constant. Up to num_threads are created on demand and then run until shutdown. + explicit ThreadPoolImpl(size_t num_threads); + + /// queue_size - maximum number of running plus scheduled jobs. It can be greater than num_threads. Zero means unlimited. + ThreadPoolImpl(size_t num_threads, size_t queue_size); + + /// Add new job. Locks until number of active jobs is less than maximum or exception in one of threads was thrown. + /// If an exception in some thread was thrown, method silently returns, and exception will be rethrown only on call to 'wait' function. + /// Priority: greater is higher. + void schedule(Job job, int priority = 0); + + /// Wait for all currently active jobs to be done. + /// You may call schedule and wait many times in arbitary order. + /// If any thread was throw an exception, first exception will be rethrown from this method, + /// and exception will be cleared. + void wait(); + + /// Waits for all threads. Doesn't rethrow exceptions (use 'wait' method to rethrow exceptions). + /// You should not destroy object while calling schedule or wait methods from another threads. + ~ThreadPoolImpl(); + + size_t size() const { return num_threads; } + + /// Returns number of running and scheduled jobs. + size_t active() const; + +private: + mutable std::mutex mutex; + std::condition_variable job_finished; + std::condition_variable new_job_or_shutdown; + + const size_t num_threads; + const size_t queue_size; + + size_t active_jobs = 0; + bool shutdown = false; + + struct JobWithPriority + { + Job job; + int priority; + + JobWithPriority(Job job, int priority) + : job(job), priority(priority) {} + + bool operator< (const JobWithPriority & rhs) const + { + return priority < rhs.priority; + } + }; + + std::priority_queue jobs; + std::vector threads; + std::exception_ptr first_exception; + + + void worker(); + + void finalize(); +}; + + +using FreeThreadPool = ThreadPoolImpl; + +class GlobalThreadPool : public FreeThreadPool, public ext::singleton +{ +public: + GlobalThreadPool() : FreeThreadPool(10000) {} /// TODO: global blocking limit may lead to deadlocks. +}; + +class ThreadFromGlobalPool +{ +public: + ThreadFromGlobalPool() {} + + ThreadFromGlobalPool(std::function func) + { + mutex = std::make_unique(); + /// The function object must be copyable, so we wrap lock_guard in shared_ptr. + GlobalThreadPool::instance().schedule([lock = std::make_shared>(*mutex), func = std::move(func)] { func(); }); + } + + ThreadFromGlobalPool(ThreadFromGlobalPool && rhs) + { + *this = std::move(rhs); + } + + ThreadFromGlobalPool & operator=(ThreadFromGlobalPool && rhs) + { + if (mutex) + std::terminate(); + mutex = std::move(rhs.mutex); + return *this; + } + + ~ThreadFromGlobalPool() + { + if (mutex) + std::terminate(); + } + + void join() + { + { + std::lock_guard lock(*mutex); + } + mutex.reset(); + } +private: + std::unique_ptr mutex; /// Object must be moveable. +}; + +using ThreadPool = ThreadPoolImpl; + + +/// Allows to save first catched exception in jobs and postpone its rethrow. +class ExceptionHandler +{ +public: + void setException(std::exception_ptr && exception); + void throwIfException(); + +private: + std::exception_ptr first_exception; + std::mutex mutex; +}; + +ThreadPool::Job createExceptionHandledJob(ThreadPool::Job job, ExceptionHandler & handler); diff --git a/dbms/src/Common/tests/multi_version.cpp b/dbms/src/Common/tests/multi_version.cpp new file mode 100644 index 00000000000..ee90a79801b --- /dev/null +++ b/dbms/src/Common/tests/multi_version.cpp @@ -0,0 +1,56 @@ +#include +#include +#include +#include +#include +#include + + +using T = std::string; +using MV = MultiVersion; +using Results = std::vector; + + +void thread1(MV & x, T & result) +{ + MV::Version v = x.get(); + result = *v; +} + +void thread2(MV & x, const char * result) +{ + x.set(std::make_unique(result)); +} + + +int main(int argc, char ** argv) +{ + try + { + const char * s1 = "Hello!"; + const char * s2 = "Goodbye!"; + + size_t n = 1000; + MV x(std::make_unique(s1)); + Results results(n); + + ThreadPool tp(8); + for (size_t i = 0; i < n; ++i) + { + tp.schedule(std::bind(thread1, std::ref(x), std::ref(results[i]))); + tp.schedule(std::bind(thread2, std::ref(x), (rand() % 2) ? s1 : s2)); + } + tp.wait(); + + for (size_t i = 0; i < n; ++i) + std::cerr << results[i] << " "; + std::cerr << std::endl; + } + catch (const Poco::Exception & e) + { + std::cerr << e.message() << std::endl; + throw; + } + + return 0; +} From 5a50a4fe2140c02e20694dd321579d471f9a9994 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 14 Jan 2019 19:27:28 +0300 Subject: [PATCH 011/324] Fix aggregate function low cardinality array argument. #4038 --- .../DataStreams/NativeBlockOutputStream.cpp | 6 +- dbms/src/DataTypes/DataTypeLowCardinality.h | 4 +- .../DataTypeLowCardinalityHelpers.cpp | 62 +++++++++++++------ dbms/src/Functions/IFunction.cpp | 12 ++-- dbms/src/Interpreters/Aggregator.cpp | 10 +-- dbms/src/Interpreters/Join.cpp | 37 ++++++++--- 6 files changed, 91 insertions(+), 40 deletions(-) diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index 11c3944afbb..4c0972af559 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -101,8 +101,10 @@ void NativeBlockOutputStream::write(const Block & block) /// Send data to old clients without low cardinality type. if (remove_low_cardinality || (client_revision && client_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE)) { - column.column = recursiveRemoveLowCardinality(column.column); - column.type = recursiveRemoveLowCardinality(column.type); + if (auto col = recursiveRemoveLowCardinality(column.column.get())) + column.column = col; + if (auto type = recursiveRemoveLowCardinality(column.type.get())) + column.type = type; } /// Name diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.h b/dbms/src/DataTypes/DataTypeLowCardinality.h index 5744419bf01..74faf038ac8 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.h +++ b/dbms/src/DataTypes/DataTypeLowCardinality.h @@ -165,10 +165,10 @@ private: DataTypePtr removeLowCardinality(const DataTypePtr & type); /// Remove LowCardinality recursively from all nested types. -DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type); +DataTypePtr recursiveRemoveLowCardinality(const IDataType * type); /// Remove LowCardinality recursively from all nested columns. -ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column); +ColumnPtr recursiveRemoveLowCardinality(const IColumn * column); /// Convert column of type from_type to type to_type by converting nested LowCardinality columns. ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type); diff --git a/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp b/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp index 215b21f7994..2b17f24969e 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp @@ -16,19 +16,31 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; } -DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type) +DataTypePtr recursiveRemoveLowCardinality(const IDataType * type) { if (!type) - return type; + return nullptr; - if (const auto * array_type = typeid_cast(type.get())) - return std::make_shared(recursiveRemoveLowCardinality(array_type->getNestedType())); + if (const auto * array_type = typeid_cast(type)) + if (auto nested = recursiveRemoveLowCardinality(array_type->getNestedType().get())) + return std::make_shared(nested); - if (const auto * tuple_type = typeid_cast(type.get())) + if (const auto * tuple_type = typeid_cast(type)) { DataTypes elements = tuple_type->getElements(); + bool has_removed = false; + for (auto & element : elements) - element = recursiveRemoveLowCardinality(element); + { + if (auto removed = recursiveRemoveLowCardinality(element.get())) + { + element = removed; + has_removed = true; + } + } + + if (!has_removed) + return nullptr; if (tuple_type->haveExplicitNames()) return std::make_shared(elements, tuple_type->getElementNames()); @@ -36,35 +48,49 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type) return std::make_shared(elements); } - if (const auto * low_cardinality_type = typeid_cast(type.get())) + if (const auto * low_cardinality_type = typeid_cast(type)) return low_cardinality_type->getDictionaryType(); - return type; + return nullptr; } -ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column) +ColumnPtr recursiveRemoveLowCardinality(const IColumn * column) { if (!column) - return column; + return nullptr; - if (const auto * column_array = typeid_cast(column.get())) - return ColumnArray::create(recursiveRemoveLowCardinality(column_array->getDataPtr()), column_array->getOffsetsPtr()); + if (const auto * column_array = typeid_cast(column)) + if (auto nested = recursiveRemoveLowCardinality(&column_array->getData())) + return ColumnArray::create(nested, column_array->getOffsetsPtr()); - if (const auto * column_const = typeid_cast(column.get())) - return ColumnConst::create(recursiveRemoveLowCardinality(column_const->getDataColumnPtr()), column_const->size()); + if (const auto * column_const = typeid_cast(column)) + if (auto nested = recursiveRemoveLowCardinality(&column_const->getDataColumn())) + return ColumnConst::create(nested, column_const->size()); - if (const auto * column_tuple = typeid_cast(column.get())) + if (const auto * column_tuple = typeid_cast(column)) { Columns columns = column_tuple->getColumns(); + bool removed_any = false; + for (auto & element : columns) - element = recursiveRemoveLowCardinality(element); + { + if (auto nested = recursiveRemoveLowCardinality(element.get())) + { + element = nested; + removed_any = true; + } + } + + if (!removed_any) + return nullptr; + return ColumnTuple::create(columns); } - if (const auto * column_low_cardinality = typeid_cast(column.get())) + if (const auto * column_low_cardinality = typeid_cast(column)) return column_low_cardinality->convertToFullColumn(); - return column; + return nullptr; } ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index ac5d1122e4a..5c753ed85fc 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -385,8 +385,10 @@ static void convertLowCardinalityColumnsToFull(Block & block, const ColumnNumber { ColumnWithTypeAndName & column = block.getByPosition(arg); - column.column = recursiveRemoveLowCardinality(column.column); - column.type = recursiveRemoveLowCardinality(column.type); + if (auto col = recursiveRemoveLowCardinality(column.column.get())) + column.column = col; + if (auto type = recursiveRemoveLowCardinality(column.type.get())) + column.type = type; } } @@ -599,8 +601,10 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar for (auto & arg : args_without_low_cardinality) { - arg.column = recursiveRemoveLowCardinality(arg.column); - arg.type = recursiveRemoveLowCardinality(arg.type); + if (auto column = recursiveRemoveLowCardinality(arg.column.get())) + arg.column = column; + if (auto type = recursiveRemoveLowCardinality(arg.type.get())) + arg.type = type; } auto type_without_low_cardinality = getReturnTypeWithoutLowCardinality(args_without_low_cardinality); diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 145ce98dbbc..91d85cd45d8 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -768,11 +768,11 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re materialized_columns.push_back(block.safeGetByPosition(params.keys[i]).column->convertToFullColumnIfConst()); key_columns[i] = materialized_columns.back().get(); - if (const auto * low_cardinality_column = typeid_cast(key_columns[i])) + if (!result.isLowCardinality()) { - if (!result.isLowCardinality()) + if (auto column = recursiveRemoveLowCardinality(key_columns[i])) { - materialized_columns.push_back(low_cardinality_column->convertToFullColumn()); + materialized_columns.emplace_back(std::move(column)); key_columns[i] = materialized_columns.back().get(); } } @@ -788,9 +788,9 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re materialized_columns.push_back(block.safeGetByPosition(params.aggregates[i].arguments[j]).column->convertToFullColumnIfConst()); aggregate_columns[i][j] = materialized_columns.back().get(); - if (auto * col_low_cardinality = typeid_cast(aggregate_columns[i][j])) + if (auto column = recursiveRemoveLowCardinality(aggregate_columns[i][j])) { - materialized_columns.push_back(col_low_cardinality->convertToFullColumn()); + materialized_columns.emplace_back(std::move(column)); aggregate_columns[i][j] = materialized_columns.back().get(); } } diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 8783d16c3c1..2f0bae96104 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -253,12 +253,16 @@ void Join::setSampleBlock(const Block & block) size_t keys_size = key_names_right.size(); ColumnRawPtrs key_columns(keys_size); - Columns materialized_columns(keys_size); + Columns materialized_columns; for (size_t i = 0; i < keys_size; ++i) { - materialized_columns[i] = recursiveRemoveLowCardinality(block.getByName(key_names_right[i]).column); - key_columns[i] = materialized_columns[i].get(); + key_columns[i] = block.getByName(key_names_right[i]).column.get(); + if (auto col = recursiveRemoveLowCardinality(key_columns[i])) + { + materialized_columns.emplace_back(std::move(col)); + key_columns[i] = materialized_columns[i].get(); + } /// We will join only keys, where all components are not NULL. if (key_columns[i]->isColumnNullable()) @@ -278,8 +282,10 @@ void Join::setSampleBlock(const Block & block) if (key_names_right.end() != std::find(key_names_right.begin(), key_names_right.end(), name)) { auto & col = sample_block_with_columns_to_add.getByPosition(pos); - col.column = recursiveRemoveLowCardinality(col.column); - col.type = recursiveRemoveLowCardinality(col.type); + if (auto column = recursiveRemoveLowCardinality(col.column.get())) + col.column = column; + if (auto type = recursiveRemoveLowCardinality(col.type.get())) + col.type = type; sample_block_with_keys.insert(col); sample_block_with_columns_to_add.erase(pos); } @@ -429,7 +435,9 @@ bool Join::insertFromBlock(const Block & block) /// Memoize key columns to work. for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_right[i]).column->convertToFullColumnIfConst())); + materialized_columns.emplace_back(block.getByName(key_names_right[i]).column->convertToFullColumnIfConst()); + if (auto col = recursiveRemoveLowCardinality(materialized_columns.back().get())) + materialized_columns.back() = col; key_columns[i] = materialized_columns.back().get(); } @@ -667,7 +675,9 @@ void Join::joinBlockImpl( /// Memoize key columns to work with. for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_left[i]).column->convertToFullColumnIfConst())); + materialized_columns.emplace_back(block.getByName(key_names_left[i]).column->convertToFullColumnIfConst()); + if (auto col = recursiveRemoveLowCardinality(materialized_columns.back().get())) + materialized_columns.back() = col; key_columns[i] = materialized_columns.back().get(); } @@ -868,8 +878,17 @@ void Join::checkTypesOfKeys(const Block & block_left, const Names & key_names_le { /// Compare up to Nullability. - DataTypePtr left_type = removeNullable(recursiveRemoveLowCardinality(block_left.getByName(key_names_left[i]).type)); - DataTypePtr right_type = removeNullable(recursiveRemoveLowCardinality(block_right.getByName(key_names_right[i]).type)); + DataTypePtr left_type = block_left.getByName(key_names_left[i]).type; + DataTypePtr right_type = block_right.getByName(key_names_right[i]).type; + + if (auto type = recursiveRemoveLowCardinality(left_type.get())) + left_type = type; + + if (auto type = recursiveRemoveLowCardinality(right_type.get())) + right_type = type; + + left_type = removeNullable(left_type); + right_type = removeNullable(right_type); if (!left_type->equals(*right_type)) throw Exception("Type mismatch of columns to JOIN by: " From f6b9b063071662f1c391ee95ff16a30c89069010 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Jan 2019 22:22:09 +0300 Subject: [PATCH 012/324] Attempt to implemnt global thread pool #4018 --- dbms/programs/server/MetricsTransmitter.h | 3 +- dbms/src/Common/Config/ConfigReloader.cpp | 2 +- dbms/src/Common/Config/ConfigReloader.h | 3 +- dbms/src/Common/CurrentThread.cpp | 2 +- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Common/MemoryTracker.cpp | 9 +- dbms/src/Common/ThreadPool.cpp | 126 +++++++++++++----- dbms/src/Common/ThreadPool.h | 82 +++++++++--- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 +- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 5 +- dbms/src/Common/tests/multi_version.cpp | 2 +- dbms/src/Core/BackgroundSchedulePool.cpp | 6 +- dbms/src/Core/BackgroundSchedulePool.h | 6 +- .../src/DataStreams/ParallelInputsProcessor.h | 5 +- .../ExecutableDictionarySource.cpp | 3 +- dbms/src/IO/AIOContextPool.h | 3 +- dbms/src/Interpreters/AsynchronousMetrics.h | 3 +- dbms/src/Interpreters/Compiler.cpp | 55 ++++---- dbms/src/Interpreters/Context.h | 3 +- dbms/src/Interpreters/DDLWorker.cpp | 2 +- dbms/src/Interpreters/DDLWorker.h | 3 +- .../src/Interpreters/EmbeddedDictionaries.cpp | 2 +- dbms/src/Interpreters/EmbeddedDictionaries.h | 3 +- dbms/src/Interpreters/ExternalLoader.cpp | 2 +- dbms/src/Interpreters/ExternalLoader.h | 3 +- dbms/src/Interpreters/SystemLog.h | 5 +- .../Storages/Distributed/DirectoryMonitor.h | 3 +- .../MergeTree/BackgroundProcessingPool.cpp | 4 +- .../MergeTree/BackgroundProcessingPool.h | 4 +- dbms/src/Storages/StorageBuffer.cpp | 2 +- dbms/src/Storages/StorageBuffer.h | 3 +- libs/libdaemon/src/BaseDaemon.cpp | 2 - 32 files changed, 248 insertions(+), 113 deletions(-) diff --git a/dbms/programs/server/MetricsTransmitter.h b/dbms/programs/server/MetricsTransmitter.h index e85113ad141..fd3853a7a9e 100644 --- a/dbms/programs/server/MetricsTransmitter.h +++ b/dbms/programs/server/MetricsTransmitter.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -46,7 +47,7 @@ private: bool quit = false; std::mutex mutex; std::condition_variable cond; - std::thread thread{&MetricsTransmitter::run, this}; + ThreadFromGlobalPool thread{&MetricsTransmitter::run, this}; static constexpr auto profile_events_path_prefix = "ClickHouse.ProfileEvents."; static constexpr auto current_metrics_path_prefix = "ClickHouse.Metrics."; diff --git a/dbms/src/Common/Config/ConfigReloader.cpp b/dbms/src/Common/Config/ConfigReloader.cpp index ed6fad4d42c..063fbec8e5b 100644 --- a/dbms/src/Common/Config/ConfigReloader.cpp +++ b/dbms/src/Common/Config/ConfigReloader.cpp @@ -33,7 +33,7 @@ ConfigReloader::ConfigReloader( void ConfigReloader::start() { - thread = std::thread(&ConfigReloader::run, this); + thread = ThreadFromGlobalPool(&ConfigReloader::run, this); } diff --git a/dbms/src/Common/Config/ConfigReloader.h b/dbms/src/Common/Config/ConfigReloader.h index ca4c97c5aee..c0904422b39 100644 --- a/dbms/src/Common/Config/ConfigReloader.h +++ b/dbms/src/Common/Config/ConfigReloader.h @@ -1,6 +1,7 @@ #pragma once #include "ConfigProcessor.h" +#include #include #include #include @@ -81,7 +82,7 @@ private: Updater updater; std::atomic quit{false}; - std::thread thread; + ThreadFromGlobalPool thread; /// Locked inside reloadIfNewer. std::mutex reload_mutex; diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index 8c05c91bac3..c3e0cae9571 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -34,7 +34,7 @@ ThreadStatus & CurrentThread::get() ProfileEvents::Counters & CurrentThread::getProfileEvents() { - return get().performance_counters; + return current_thread ? get().performance_counters : ProfileEvents::global_counters; } MemoryTracker & CurrentThread::getMemoryTracker() diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 13ea9e4744a..eb52b6ff7e3 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -408,6 +408,7 @@ namespace ErrorCodes extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE = 431; extern const int UNKNOWN_CODEC = 432; extern const int ILLEGAL_CODEC_PARAMETER = 433; + extern const int CANNOT_SCHEDULE_TASK = 434; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index f7c2eb0ef78..6a997e3b19a 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -190,17 +190,20 @@ namespace CurrentMemoryTracker { void alloc(Int64 size) { - DB::CurrentThread::getMemoryTracker().alloc(size); + if (DB::current_thread) + DB::CurrentThread::getMemoryTracker().alloc(size); } void realloc(Int64 old_size, Int64 new_size) { - DB::CurrentThread::getMemoryTracker().alloc(new_size - old_size); + if (DB::current_thread) + DB::CurrentThread::getMemoryTracker().alloc(new_size - old_size); } void free(Int64 size) { - DB::CurrentThread::getMemoryTracker().free(size); + if (DB::current_thread) + DB::CurrentThread::getMemoryTracker().free(size); } } diff --git a/dbms/src/Common/ThreadPool.cpp b/dbms/src/Common/ThreadPool.cpp index e3f03e18a46..487bd6fd66d 100644 --- a/dbms/src/Common/ThreadPool.cpp +++ b/dbms/src/Common/ThreadPool.cpp @@ -1,44 +1,103 @@ -#include +#include +#include + #include +#include + + +namespace DB +{ + namespace ErrorCodes + { + extern const int CANNOT_SCHEDULE_TASK; + } +} template -ThreadPoolImpl::ThreadPoolImpl(size_t num_threads) - : ThreadPoolImpl(num_threads, num_threads) +ThreadPoolImpl::ThreadPoolImpl(size_t max_threads) + : ThreadPoolImpl(max_threads, max_threads, max_threads) { } template -ThreadPoolImpl::ThreadPoolImpl(size_t num_threads, size_t queue_size) - : num_threads(num_threads), queue_size(queue_size) +ThreadPoolImpl::ThreadPoolImpl(size_t max_threads, size_t max_free_threads, size_t queue_size) + : max_threads(max_threads), max_free_threads(max_free_threads), queue_size(queue_size) { - threads.reserve(num_threads); +} + +template +template +ReturnType ThreadPoolImpl::scheduleImpl(Job job, int priority, std::optional wait_microseconds) +{ + auto on_error = [] + { + if constexpr (std::is_same_v) + throw DB::Exception("Cannot schedule a task", DB::ErrorCodes::CANNOT_SCHEDULE_TASK); + else + return false; + }; + + { + std::unique_lock lock(mutex); + + auto pred = [this] { return !queue_size || scheduled_jobs < queue_size || shutdown; }; + + if (wait_microseconds) + { + if (!job_finished.wait_for(lock, std::chrono::microseconds(*wait_microseconds), pred)) + return on_error(); + } + else + job_finished.wait(lock, pred); + + if (shutdown) + return on_error(); + + jobs.emplace(std::move(job), priority); + ++scheduled_jobs; + + if (threads.size() < std::min(max_threads, scheduled_jobs)) + { + threads.emplace_front(); + try + { + threads.front() = Thread([this, it = threads.begin()] { worker(it); }); + } + catch (...) + { + threads.pop_front(); + } + } + } + new_job_or_shutdown.notify_one(); + return ReturnType(true); } template void ThreadPoolImpl::schedule(Job job, int priority) { - { - std::unique_lock lock(mutex); - job_finished.wait(lock, [this] { return !queue_size || active_jobs < queue_size || shutdown; }); - if (shutdown) - return; + scheduleImpl(std::move(job), priority, std::nullopt); +} - jobs.emplace(std::move(job), priority); - ++active_jobs; +template +bool ThreadPoolImpl::trySchedule(Job job, int priority, uint64_t wait_microseconds) +{ + return scheduleImpl(std::move(job), priority, wait_microseconds); +} - if (threads.size() < std::min(num_threads, active_jobs)) - threads.emplace_back([this] { worker(); }); - } - new_job_or_shutdown.notify_one(); +template +void ThreadPoolImpl::scheduleOrThrow(Job job, int priority, uint64_t wait_microseconds) +{ + scheduleImpl(std::move(job), priority, wait_microseconds); } template void ThreadPoolImpl::wait() { { - std::unique_lock lock(mutex); - job_finished.wait(lock, [this] { return active_jobs == 0; }); + std::unique_lock lock(mutex); + job_finished.wait(lock, [this] { return scheduled_jobs == 0; }); if (first_exception) { @@ -59,7 +118,7 @@ template void ThreadPoolImpl::finalize() { { - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); shutdown = true; } @@ -74,12 +133,12 @@ void ThreadPoolImpl::finalize() template size_t ThreadPoolImpl::active() const { - std::unique_lock lock(mutex); - return active_jobs; + std::unique_lock lock(mutex); + return scheduled_jobs; } template -void ThreadPoolImpl::worker() +void ThreadPoolImpl::worker(typename std::list::iterator thread_it) { while (true) { @@ -87,7 +146,7 @@ void ThreadPoolImpl::worker() bool need_shutdown = false; { - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); }); need_shutdown = shutdown; @@ -111,11 +170,11 @@ void ThreadPoolImpl::worker() catch (...) { { - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); if (!first_exception) first_exception = std::current_exception(); shutdown = true; - --active_jobs; + --scheduled_jobs; } job_finished.notify_all(); new_job_or_shutdown.notify_all(); @@ -124,8 +183,15 @@ void ThreadPoolImpl::worker() } { - std::unique_lock lock(mutex); - --active_jobs; + std::unique_lock lock(mutex); + --scheduled_jobs; + + if (threads.size() > scheduled_jobs + max_free_threads) + { + threads.erase(thread_it); + job_finished.notify_all(); + return; + } } job_finished.notify_all(); @@ -139,14 +205,14 @@ template class ThreadPoolImpl; void ExceptionHandler::setException(std::exception_ptr && exception) { - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); if (!first_exception) first_exception = std::move(exception); } void ExceptionHandler::throwIfException() { - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); if (first_exception) std::rethrow_exception(first_exception); } diff --git a/dbms/src/Common/ThreadPool.h b/dbms/src/Common/ThreadPool.h index a8cf84dd7b5..c0ab07fdcf5 100644 --- a/dbms/src/Common/ThreadPool.h +++ b/dbms/src/Common/ThreadPool.h @@ -6,15 +6,23 @@ #include #include #include -#include +#include +#include #include +#include + /** Very simple thread pool similar to boost::threadpool. * Advantages: * - catches exceptions and rethrows on wait. + * + * This thread pool can be used as a task queue. + * For example, you can create a thread pool with 10 threads (and queue of size 10) and schedule 1000 tasks + * - in this case you will be blocked to keep 10 tasks in fly. + * + * Thread: std::thread or something with identical interface. */ - template class ThreadPoolImpl { @@ -22,16 +30,22 @@ public: using Job = std::function; /// Size is constant. Up to num_threads are created on demand and then run until shutdown. - explicit ThreadPoolImpl(size_t num_threads); + explicit ThreadPoolImpl(size_t max_threads); - /// queue_size - maximum number of running plus scheduled jobs. It can be greater than num_threads. Zero means unlimited. - ThreadPoolImpl(size_t num_threads, size_t queue_size); + /// queue_size - maximum number of running plus scheduled jobs. It can be greater than max_threads. Zero means unlimited. + ThreadPoolImpl(size_t max_threads, size_t max_free_threads, size_t queue_size); - /// Add new job. Locks until number of active jobs is less than maximum or exception in one of threads was thrown. + /// Add new job. Locks until number of scheduled jobs is less than maximum or exception in one of threads was thrown. /// If an exception in some thread was thrown, method silently returns, and exception will be rethrown only on call to 'wait' function. /// Priority: greater is higher. void schedule(Job job, int priority = 0); + /// Wait for specified amount of time and schedule a job or return false. + bool trySchedule(Job job, int priority = 0, uint64_t wait_microseconds = 0); + + /// Wait for specified amount of time and schedule a job or throw an exception. + void scheduleOrThrow(Job job, int priority = 0, uint64_t wait_microseconds = 0); + /// Wait for all currently active jobs to be done. /// You may call schedule and wait many times in arbitary order. /// If any thread was throw an exception, first exception will be rethrown from this method, @@ -42,8 +56,6 @@ public: /// You should not destroy object while calling schedule or wait methods from another threads. ~ThreadPoolImpl(); - size_t size() const { return num_threads; } - /// Returns number of running and scheduled jobs. size_t active() const; @@ -52,10 +64,11 @@ private: std::condition_variable job_finished; std::condition_variable new_job_or_shutdown; - const size_t num_threads; + const size_t max_threads; + const size_t max_free_threads; const size_t queue_size; - size_t active_jobs = 0; + size_t scheduled_jobs = 0; bool shutdown = false; struct JobWithPriority @@ -73,34 +86,65 @@ private: }; std::priority_queue jobs; - std::vector threads; + std::list threads; std::exception_ptr first_exception; - void worker(); + template + ReturnType scheduleImpl(Job job, int priority, std::optional wait_microseconds); + + void worker(typename std::list::iterator thread_it); void finalize(); }; +/// ThreadPool with std::thread for threads. using FreeThreadPool = ThreadPoolImpl; + +/** Global ThreadPool that can be used as a singleton. + * Why it is needed? + * + * Linux can create and destroy about 100 000 threads per second (quite good). + * With simple ThreadPool (based on mutex and condvar) you can assign about 200 000 tasks per second + * - not much difference comparing to not using a thread pool at all. + * + * But if you reuse OS threads instead of creating and destroying them, several benefits exist: + * - allocator performance will usually be better due to reuse of thread local caches, especially for jemalloc: + * https://github.com/jemalloc/jemalloc/issues/1347 + * - address sanitizer and thread sanitizer will not fail due to global limit on number of created threads. + * - program will work faster in gdb; + */ class GlobalThreadPool : public FreeThreadPool, public ext::singleton { public: - GlobalThreadPool() : FreeThreadPool(10000) {} /// TODO: global blocking limit may lead to deadlocks. + GlobalThreadPool() : FreeThreadPool(10000, 1000, 10000) {} }; + +/** Looks like std::thread but allocates threads in GlobalThreadPool. + * Also holds ThreadStatus for ClickHouse. + */ class ThreadFromGlobalPool { public: ThreadFromGlobalPool() {} - ThreadFromGlobalPool(std::function func) + template + explicit ThreadFromGlobalPool(Function && func, Args &&... args) { mutex = std::make_unique(); + /// The function object must be copyable, so we wrap lock_guard in shared_ptr. - GlobalThreadPool::instance().schedule([lock = std::make_shared>(*mutex), func = std::move(func)] { func(); }); + GlobalThreadPool::instance().scheduleOrThrow([ + lock = std::make_shared>(*mutex), + func = std::forward(func), + args = std::make_tuple(std::forward(args)...)] + { + DB::ThreadStatus thread_status; + std::apply(func, args); + }); } ThreadFromGlobalPool(ThreadFromGlobalPool && rhs) @@ -129,10 +173,18 @@ public: } mutex.reset(); } + + bool joinable() const + { + return static_cast(mutex); + } + private: std::unique_ptr mutex; /// Object must be moveable. }; + +/// Recommended thread pool for the case when multiple thread pools are created and destroyed. using ThreadPool = ThreadPoolImpl; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 9626a54aa20..ac049bcb8e5 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -853,8 +853,8 @@ ZooKeeper::ZooKeeper( if (!auth_scheme.empty()) sendAuth(auth_scheme, auth_data); - send_thread = std::thread([this] { sendThread(); }); - receive_thread = std::thread([this] { receiveThread(); }); + send_thread = ThreadFromGlobalPool([this] { sendThread(); }); + receive_thread = ThreadFromGlobalPool([this] { receiveThread(); }); ProfileEvents::increment(ProfileEvents::ZooKeeperInit); } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index c93f13b9351..e5da9ea48fe 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -209,8 +210,8 @@ private: Watches watches; std::mutex watches_mutex; - std::thread send_thread; - std::thread receive_thread; + ThreadFromGlobalPool send_thread; + ThreadFromGlobalPool receive_thread; void connect( const Addresses & addresses, diff --git a/dbms/src/Common/tests/multi_version.cpp b/dbms/src/Common/tests/multi_version.cpp index ee90a79801b..0937e597e2d 100644 --- a/dbms/src/Common/tests/multi_version.cpp +++ b/dbms/src/Common/tests/multi_version.cpp @@ -23,7 +23,7 @@ void thread2(MV & x, const char * result) } -int main(int argc, char ** argv) +int main(int, char **) { try { diff --git a/dbms/src/Core/BackgroundSchedulePool.cpp b/dbms/src/Core/BackgroundSchedulePool.cpp index 5da499e5ae9..0493e13b2b9 100644 --- a/dbms/src/Core/BackgroundSchedulePool.cpp +++ b/dbms/src/Core/BackgroundSchedulePool.cpp @@ -161,9 +161,9 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size) threads.resize(size); for (auto & thread : threads) - thread = std::thread([this] { threadFunction(); }); + thread = ThreadFromGlobalPool([this] { threadFunction(); }); - delayed_thread = std::thread([this] { delayExecutionThreadFunction(); }); + delayed_thread = ThreadFromGlobalPool([this] { delayExecutionThreadFunction(); }); } @@ -181,7 +181,7 @@ BackgroundSchedulePool::~BackgroundSchedulePool() delayed_thread.join(); LOG_TRACE(&Logger::get("BackgroundSchedulePool"), "Waiting for threads to finish."); - for (std::thread & thread : threads) + for (auto & thread : threads) thread.join(); } catch (...) diff --git a/dbms/src/Core/BackgroundSchedulePool.h b/dbms/src/Core/BackgroundSchedulePool.h index ba23d93733f..7b75d9459ba 100644 --- a/dbms/src/Core/BackgroundSchedulePool.h +++ b/dbms/src/Core/BackgroundSchedulePool.h @@ -13,6 +13,8 @@ #include #include #include +#include + namespace DB { @@ -119,7 +121,7 @@ public: ~BackgroundSchedulePool(); private: - using Threads = std::vector; + using Threads = std::vector; void threadFunction(); void delayExecutionThreadFunction(); @@ -141,7 +143,7 @@ private: std::condition_variable wakeup_cond; std::mutex delayed_tasks_mutex; /// Thread waiting for next delayed task. - std::thread delayed_thread; + ThreadFromGlobalPool delayed_thread; /// Tasks ordered by scheduled time. DelayedTasks delayed_tasks; diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index ba086b98939..a83c2ca1e56 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -13,6 +13,7 @@ #include #include #include +#include /** Allows to process multiple block input streams (sources) in parallel, using specified number of threads. @@ -306,8 +307,8 @@ private: Handler & handler; - /// Streams. - using ThreadsData = std::vector; + /// Threads. + using ThreadsData = std::vector; ThreadsData threads; /** A set of available sources that are not currently processed by any thread. diff --git a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp index 376153bd0e9..028e0452fff 100644 --- a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include "DictionarySourceFactory.h" #include "DictionarySourceHelpers.h" @@ -165,7 +166,7 @@ namespace BlockInputStreamPtr stream; std::unique_ptr command; std::packaged_task task; - std::thread thread; + ThreadFromGlobalPool thread; bool wait_called = false; }; diff --git a/dbms/src/IO/AIOContextPool.h b/dbms/src/IO/AIOContextPool.h index 64d01a0f45b..ca92e14b6ed 100644 --- a/dbms/src/IO/AIOContextPool.h +++ b/dbms/src/IO/AIOContextPool.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -32,7 +33,7 @@ class AIOContextPool : public ext::singleton std::map> promises; std::atomic cancelled{false}; - std::thread io_completion_monitor{&AIOContextPool::doMonitor, this}; + ThreadFromGlobalPool io_completion_monitor{&AIOContextPool::doMonitor, this}; ~AIOContextPool(); diff --git a/dbms/src/Interpreters/AsynchronousMetrics.h b/dbms/src/Interpreters/AsynchronousMetrics.h index ceafc2af586..8ccefb9e930 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.h +++ b/dbms/src/Interpreters/AsynchronousMetrics.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -43,7 +44,7 @@ private: Container container; mutable std::mutex container_mutex; - std::thread thread; + ThreadFromGlobalPool thread; void run(); void update(); diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp index 8a60b24a24b..9b0a8371f09 100644 --- a/dbms/src/Interpreters/Compiler.cpp +++ b/dbms/src/Interpreters/Compiler.cpp @@ -142,40 +142,37 @@ SharedLibraryPtr Compiler::getOrCount( { /// The min_count_to_compile value of zero indicates the need for synchronous compilation. - /// Are there any free threads? - if (min_count_to_compile == 0 || pool.active() < pool.size()) + /// Indicates that the library is in the process of compiling. + libraries[hashed_key] = nullptr; + + LOG_INFO(log, "Compiling code " << file_name << ", key: " << key); + + if (min_count_to_compile == 0) { - /// Indicates that the library is in the process of compiling. - libraries[hashed_key] = nullptr; - - LOG_INFO(log, "Compiling code " << file_name << ", key: " << key); - - if (min_count_to_compile == 0) { - { - ext::unlock_guard unlock(mutex); - compile(hashed_key, file_name, additional_compiler_flags, get_code, on_ready); - } + ext::unlock_guard unlock(mutex); + compile(hashed_key, file_name, additional_compiler_flags, get_code, on_ready); + } - return libraries[hashed_key]; - } - else - { - pool.schedule([=] - { - try - { - compile(hashed_key, file_name, additional_compiler_flags, get_code, on_ready); - } - catch (...) - { - tryLogCurrentException("Compiler"); - } - }); - } + return libraries[hashed_key]; } else - LOG_INFO(log, "All threads are busy."); + { + bool res = pool.trySchedule([=] + { + try + { + compile(hashed_key, file_name, additional_compiler_flags, get_code, on_ready); + } + catch (...) + { + tryLogCurrentException("Compiler"); + } + }); + + if (!res) + LOG_INFO(log, "All threads are busy."); + } } return nullptr; diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index ca90073436a..749c2ae40d5 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -521,7 +522,7 @@ private: std::mutex mutex; std::condition_variable cond; std::atomic quit{false}; - std::thread thread{&SessionCleaner::run, this}; + ThreadFromGlobalPool thread{&SessionCleaner::run, this}; }; } diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 54fcffbea2a..730e37d9bd3 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -241,7 +241,7 @@ DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_, const event_queue_updated = std::make_shared(); - thread = std::thread(&DDLWorker::run, this); + thread = ThreadFromGlobalPool(&DDLWorker::run, this); } diff --git a/dbms/src/Interpreters/DDLWorker.h b/dbms/src/Interpreters/DDLWorker.h index d3872b8ac95..18714720d2d 100644 --- a/dbms/src/Interpreters/DDLWorker.h +++ b/dbms/src/Interpreters/DDLWorker.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -90,7 +91,7 @@ private: std::shared_ptr event_queue_updated; std::atomic stop_flag{false}; - std::thread thread; + ThreadFromGlobalPool thread; Int64 last_cleanup_time_seconds = 0; diff --git a/dbms/src/Interpreters/EmbeddedDictionaries.cpp b/dbms/src/Interpreters/EmbeddedDictionaries.cpp index 10f5692f6e6..60524d63cee 100644 --- a/dbms/src/Interpreters/EmbeddedDictionaries.cpp +++ b/dbms/src/Interpreters/EmbeddedDictionaries.cpp @@ -150,7 +150,7 @@ EmbeddedDictionaries::EmbeddedDictionaries( , reload_period(context_.getConfigRef().getInt("builtin_dictionaries_reload_interval", 3600)) { reloadImpl(throw_on_error); - reloading_thread = std::thread([this] { reloadPeriodically(); }); + reloading_thread = ThreadFromGlobalPool([this] { reloadPeriodically(); }); } diff --git a/dbms/src/Interpreters/EmbeddedDictionaries.h b/dbms/src/Interpreters/EmbeddedDictionaries.h index ad2dd404b3e..caa7c1cc62d 100644 --- a/dbms/src/Interpreters/EmbeddedDictionaries.h +++ b/dbms/src/Interpreters/EmbeddedDictionaries.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -41,7 +42,7 @@ private: mutable std::mutex mutex; - std::thread reloading_thread; + ThreadFromGlobalPool reloading_thread; Poco::Event destroy; diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index e4ccd9962c6..814fc5ecec2 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -72,7 +72,7 @@ void ExternalLoader::init(bool throw_on_error) reloadAndUpdate(throw_on_error); } - reloading_thread = std::thread{&ExternalLoader::reloadPeriodically, this}; + reloading_thread = ThreadFromGlobalPool{&ExternalLoader::reloadPeriodically, this}; } diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index ac672f925e3..c2ce161f0e1 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -160,7 +161,7 @@ private: std::unique_ptr config_repository; - std::thread reloading_thread; + ThreadFromGlobalPool reloading_thread; Poco::Event destroy; Logger * log; diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index 1a4283fae8e..4cb2bb76b4f 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -135,7 +136,7 @@ protected: /** In this thread, data is pulled from 'queue' and stored in 'data', and then written into table. */ - std::thread saving_thread; + ThreadFromGlobalPool saving_thread; void threadFunction(); @@ -161,7 +162,7 @@ SystemLog::SystemLog(Context & context_, log = &Logger::get("SystemLog (" + database_name + "." + table_name + ")"); data.reserve(DBMS_SYSTEM_LOG_QUEUE_SIZE); - saving_thread = std::thread([this] { threadFunction(); }); + saving_thread = ThreadFromGlobalPool([this] { threadFunction(); }); } diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.h b/dbms/src/Storages/Distributed/DirectoryMonitor.h index 484b0ac3f23..d7858d3af40 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.h +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -55,7 +56,7 @@ private: std::mutex mutex; std::condition_variable cond; Logger * log; - std::thread thread {&StorageDistributedDirectoryMonitor::run, this}; + ThreadFromGlobalPool thread{&StorageDistributedDirectoryMonitor::run, this}; }; } diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index c0911ac4d5e..b60d860ec6c 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -67,7 +67,7 @@ BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_) threads.resize(size); for (auto & thread : threads) - thread = std::thread([this] { threadFunction(); }); + thread = ThreadFromGlobalPool([this] { threadFunction(); }); } @@ -110,7 +110,7 @@ BackgroundProcessingPool::~BackgroundProcessingPool() { shutdown = true; wake_event.notify_all(); - for (std::thread & thread : threads) + for (auto & thread : threads) thread.join(); } catch (...) diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h index 4eb5d4cce56..fdf5251cb8a 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h @@ -13,6 +13,8 @@ #include #include #include +#include + namespace DB @@ -60,7 +62,7 @@ protected: friend class BackgroundProcessingPoolTaskInfo; using Tasks = std::multimap; /// key is desired next time to execute (priority). - using Threads = std::vector; + using Threads = std::vector; const size_t size; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 5d76279c95f..511364bc11f 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -420,7 +420,7 @@ void StorageBuffer::startup() << " Set apropriate system_profile to fix this."); } - flush_thread = std::thread(&StorageBuffer::flushThread, this); + flush_thread = ThreadFromGlobalPool(&StorageBuffer::flushThread, this); } diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 9992d1b49bd..85ea3f086b5 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -110,7 +111,7 @@ private: Poco::Event shutdown_event; /// Resets data by timeout. - std::thread flush_thread; + ThreadFromGlobalPool flush_thread; void flushAllBuffers(bool check_thresholds = true); /// Reset the buffer. If check_thresholds is set - resets only if thresholds are exceeded. diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index bad38c78529..e62ff31172f 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -998,8 +998,6 @@ void BaseDaemon::initialize(Application & self) } initializeTerminationAndSignalProcessing(); - - DB::CurrentThread::get(); /// TODO Why do we need this? logRevision(); for (const auto & key : DB::getMultipleKeysFromConfig(config(), "", "graphite")) From a98e822a100d5d2967afbd22c2cc19d422a499cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Jan 2019 21:39:54 +0300 Subject: [PATCH 013/324] Attempt to implemnt global thread pool #4018 --- dbms/programs/server/HTTPHandler.cpp | 1 + dbms/programs/server/Server.cpp | 4 +++- dbms/programs/server/TCPHandler.cpp | 1 + 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index d86c526784b..5881314bea7 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -647,6 +647,7 @@ void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) { setThreadName("HTTPHandler"); + ThreadStatus thread_status; Output used_output; diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 499f233ff28..176150297f2 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -129,9 +130,10 @@ std::string Server::getDefaultCorePath() const int Server::main(const std::vector & /*args*/) { Logger * log = &logger(); - UseSSL use_ssl; + ThreadStatus thread_status; + registerFunctions(); registerAggregateFunctions(); registerTableFunctions(); diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index c3dff11146e..9eccf1a819e 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -55,6 +55,7 @@ namespace ErrorCodes void TCPHandler::runImpl() { setThreadName("TCPHandler"); + ThreadStatus thread_status; connection_context = server.context(); connection_context.setSessionContext(connection_context); From b2690d8eed51bf172e3a7fffe1108186d86930b4 Mon Sep 17 00:00:00 2001 From: Odin Hultgren Van Der Horst Date: Wed, 16 Jan 2019 14:49:41 +0100 Subject: [PATCH 014/324] Added zstd support for kafka --- contrib/librdkafka-cmake/CMakeLists.txt | 1 + contrib/librdkafka-cmake/config.h | 3 +++ 2 files changed, 4 insertions(+) diff --git a/contrib/librdkafka-cmake/CMakeLists.txt b/contrib/librdkafka-cmake/CMakeLists.txt index 115c916e9f4..fb27895c3da 100644 --- a/contrib/librdkafka-cmake/CMakeLists.txt +++ b/contrib/librdkafka-cmake/CMakeLists.txt @@ -2,6 +2,7 @@ set(RDKAFKA_SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/librdkafka/src) set(SRCS ${RDKAFKA_SOURCE_DIR}/crc32c.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_zstd.c ${RDKAFKA_SOURCE_DIR}/rdaddr.c ${RDKAFKA_SOURCE_DIR}/rdavl.c ${RDKAFKA_SOURCE_DIR}/rdbuf.c diff --git a/contrib/librdkafka-cmake/config.h b/contrib/librdkafka-cmake/config.h index 2ffc5a497ae..0de2ca1a383 100644 --- a/contrib/librdkafka-cmake/config.h +++ b/contrib/librdkafka-cmake/config.h @@ -10,6 +10,7 @@ #define ENABLE_SHAREDPTR_DEBUG 0 #define ENABLE_LZ4_EXT 1 #define ENABLE_SSL 1 +//#define WITH_HDRHISTOGRAM 0 //#define ENABLE_SASL 1 #define MKL_APP_NAME "librdkafka" #define MKL_APP_DESC_ONELINE "The Apache Kafka C/C++ library" @@ -51,6 +52,8 @@ //#define WITH_PLUGINS 1 // zlib #define WITH_ZLIB 1 +// zstd +#define WITH_ZSTD 1 // WITH_SNAPPY #define WITH_SNAPPY 1 // WITH_SOCKEM From addc9a43b014702c6dd8f51ccb6e55293988cd2a Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 17 Jan 2019 18:52:29 +0300 Subject: [PATCH 015/324] remove unused IAST field --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 2 -- dbms/src/Parsers/IAST.h | 2 -- 2 files changed, 4 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 7fa74884d5a..429c257b8d5 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -345,7 +345,6 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns) ParserIdentifierWithOptionalParameters storage_p; column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0); - column_declaration->type->owned_string = type_name; columns_list->children.emplace_back(column_declaration); } @@ -369,7 +368,6 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) ParserIdentifierWithOptionalParameters storage_p; column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0); - column_declaration->type->owned_string = type_name; const auto defaults_it = columns.defaults.find(column.name); if (defaults_it != std::end(columns.defaults)) diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index 703fdfc2765..a597d40080f 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -56,8 +56,6 @@ public: ASTs children; StringRange range; - /// This pointer does not allow it to be deleted while the range refers to it. - StringPtr owned_string; SemanticPtr semantic; virtual ~IAST() = default; From d2074985734522380305e5a3e4933fe9663f845a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 Jan 2019 13:39:24 +0300 Subject: [PATCH 016/324] Added ColumnsHashing --- dbms/src/Common/ColumnsHashing.h | 881 +++++++++++++++++++++++++++ dbms/src/Interpreters/Aggregator.cpp | 173 ++---- dbms/src/Interpreters/Aggregator.h | 638 +------------------ 3 files changed, 937 insertions(+), 755 deletions(-) create mode 100644 dbms/src/Common/ColumnsHashing.h diff --git a/dbms/src/Common/ColumnsHashing.h b/dbms/src/Common/ColumnsHashing.h new file mode 100644 index 00000000000..0a6d5464341 --- /dev/null +++ b/dbms/src/Common/ColumnsHashing.h @@ -0,0 +1,881 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ColumnsHashing +{ + +/// Generic context for HashMethod. Context is shared between multiple threads, all methods must be thread-safe. +/// Is used for caching. +class HashMethodContext +{ +public: + virtual ~HashMethodContext() = default; + + struct Settings + { + size_t max_threads; + }; +}; + +using HashMethodContextPtr = std::shared_ptr; + + +template +struct MappedTraits +{ + using Type = void *; + static Type getMapped(T &) { return nullptr; } + static T & getKey(T & key) { return key; } +}; + +template +struct MappedTraits> +{ + using Type = Second *; + static Type getMapped(PairNoInit & value) { return &value.second; } + static First & getKey(PairNoInit & value) { return value.first; } +}; + +template +struct HashTableTraits +{ + using Value = typename Data::value_type; + using Mapped = typename MappedTraits::Type; + + static Mapped getMapped(Value & value) { return MappedTraits::getMapped(value); } + static auto & getKey(Value & value) { return MappedTraits::getKey(value); } +}; + +template +struct LastElementCache +{ + static constexpr bool consecutive_keys_optimization = consecutive_keys_optimization_; + using Value = typename HashTableTraits::Value; + Value value; + bool empty = true; + bool found = false; + + auto getMapped() { return HashTableTraits::getMapped(value); } + auto & getKey() { return HashTableTraits::getKey(value); } +}; + +template +struct LastElementCache +{ + static constexpr bool consecutive_keys_optimization = false; +}; + +template +inline ALWAYS_INLINE typename HashTableTraits::Value & emplaceKeyImpl( + Key key, Data & data, bool & inserted, Cache & cache [[maybe_unused]]) +{ + if constexpr (Cache::consecutive_keys_optimization) + { + if (!cache.empty && cache.found && cache.getKey() == key) + { + inserted = false; + return cache.value; + } + } + + typename Data::iterator it; + data.emplace(key, it, inserted); + auto & value = *it; + + if constexpr (Cache::consecutive_keys_optimization) + { + cache.value = value; + cache.empty = false; + cache.found = true; + } + + return value; +} + +template +inline ALWAYS_INLINE typename HashTableTraits::Mapped findKeyImpl( + Key key, Data & data, bool & found, Cache & cache [[maybe_unused]]) +{ + if constexpr (Cache::consecutive_keys_optimization) + { + if (!cache.empty && cache.getKey() == key) + { + found = cache.found; + return found ? cache.getMapped() : nullptr; + } + } + + auto it = data.find(key); + + found = it != data.end(); + auto mapped = found ? HashTableTraits::getMapped(*it) + : nullptr; + + if constexpr (Cache::consecutive_keys_optimization) + { + if (found) + cache.value = *it; + else + cache.getKey() = key; + + cache.empty = false; + cache.found = found; + } + + return mapped; +} + + +/// For the case where there is one numeric key. +template /// UInt8/16/32/64 for any type with corresponding bit width. +struct HashMethodOneNumber +{ + const char * vec; + LastElementCache last_elem_cache; + + /// If the keys of a fixed length then key_sizes contains their lengths, empty otherwise. + HashMethodOneNumber(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) + { + vec = key_columns[0]->getRawData().data; + } + + /// Creates context. Method is called once and result context is used in all threads. + static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; } + + FieldType getKey(size_t row) const { return unalignedLoad(vec + row * sizeof(FieldType)); } + + /// Emplace key into HashTable or HashMap. If Data is HashMap, returns ptr to value, otherwise nullptr. + template + ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey( + Data & data, /// HashTable + size_t row, /// From which row of the block insert the key + bool & inserted, + Arena & /*pool*/) /// For Serialized method, key may be placed in pool. + { + return HashTableTraits::getMapped(emplaceKeyImpl(getKey(row), data, inserted, last_elem_cache)); + } + + /// Find key into HashTable or HashMap. If Data is HashMap and key was found, returns ptr to value, otherwise nullptr. + template + ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & /*pool*/) + { + return findKeyImpl(getKey(row), data, found, last_elem_cache); + } + + /// Insert the key from the hash table into columns. + template + static void insertKeyIntoColumns(const Value & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) + { + static_cast(key_columns[0].get())->insertRawData(reinterpret_cast(&value.first)); + } + + /// Get hash value of row. + template + ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & /*pool*/) + { + return data.hash(getKey(row)); + } + + /// Get StringRef from value which can be inserted into column. + template + static StringRef getValueRef(const Value & value) + { + return StringRef(reinterpret_cast(&value.first), sizeof(value.first)); + } + + /// Cache last result if key was inserted. + template + ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped mapped) + { + *last_elem_cache.getMapped() = mapped; + } + +protected: + template + static ALWAYS_INLINE void onNewKey(Value & /*value*/, Arena & /*pool*/) {} +}; + + +/// For the case where there is one string key. +template +struct HashMethodString +{ + const IColumn::Offset * offsets; + const UInt8 * chars; + + LastElementCache last_elem_cache; + + HashMethodString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) + { + const IColumn & column = *key_columns[0]; + const ColumnString & column_string = static_cast(column); + offsets = column_string.getOffsets().data(); + chars = column_string.getChars().data(); + } + + static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; } + + StringRef getKey(size_t row) const { return StringRef(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1); } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row, bool & inserted, Arena & pool) + { + auto & value = emplaceKeyImpl(getKey(row), data, inserted, last_elem_cache); + if (inserted) + { + auto & key = HashTableTraits::getKey(value); + if (key.size) + key.data = pool.insert(key.data, key.size); + } + return HashTableTraits::getMapped(value); + } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & /*pool*/) + { + return findKeyImpl(getKey(row), data, found, last_elem_cache); + } + + template + static void insertKeyIntoColumns(const Value & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) + { + key_columns[0]->insertData(value.first.data, value.first.size); + } + + template + ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & /*pool*/) + { + return data.hash(getKey(row)); + } + + template + static StringRef getValueRef(const Value & value) + { + return StringRef(value.first.data, value.first.size); + } + + template + ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped mapped) + { + *last_elem_cache.getMapped() = mapped; + } + +protected: + template + static ALWAYS_INLINE void onNewKey(Value & value, Arena & pool) + { + if (value.first.size) + value.first.data = pool.insert(value.first.data, value.first.size); + } +}; + + +/// For the case where there is one fixed-length string key. +template +struct HashMethodFixedString +{ + size_t n; + const ColumnFixedString::Chars * chars; + + LastElementCache last_elem_cache; + + HashMethodFixedString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) + { + const IColumn & column = *key_columns[0]; + const ColumnFixedString & column_string = static_cast(column); + n = column_string.getN(); + chars = &column_string.getChars(); + } + + static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; } + + StringRef getKey(size_t row) const { return StringRef(&(*chars)[row * n], n); } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row, bool & inserted, Arena & pool) + { + auto & value = emplaceKeyImpl(getKey(row), data, inserted, last_elem_cache); + if (inserted) + { + auto & key = HashTableTraits::getKey(value); + key.data = pool.insert(key.data, key.size); + } + return HashTableTraits::getMapped(value); + } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & /*pool*/) + { + return findKeyImpl(getKey(row), data, found, last_elem_cache); + } + + template + static void insertKeyIntoColumns(const Value & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) + { + key_columns[0]->insertData(value.first.data, value.first.size); + } + + template + ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & /*pool*/) + { + return data.hash(getKey(row)); + } + + template + static StringRef getValueRef(const Value & value) + { + return StringRef(value.first.data, value.first.size); + } + + template + ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped mapped) + { + *last_elem_cache.getMapped() = mapped; + } + +protected: + template + static ALWAYS_INLINE void onNewKey(Value & value, Arena & pool) + { + value.first.data = pool.insert(value.first.data, value.first.size); + } +}; + + +/// Cache stores dictionaries and saved_hash per dictionary key. +class LowCardinalityDictionaryCache : public HashMethodContext +{ +public: + /// Will assume that dictionaries with same hash has the same keys. + /// Just in case, check that they have also the same size. + struct DictionaryKey + { + UInt128 hash; + UInt64 size; + + bool operator== (const DictionaryKey & other) const { return hash == other.hash && size == other.size; } + }; + + struct DictionaryKeyHash + { + size_t operator()(const DictionaryKey & key) const + { + SipHash hash; + hash.update(key.hash.low); + hash.update(key.hash.high); + hash.update(key.size); + return hash.get64(); + } + }; + + struct CachedValues + { + /// Store ptr to dictionary to be sure it won't be deleted. + ColumnPtr dictionary_holder; + /// Hashes for dictionary keys. + const UInt64 * saved_hash = nullptr; + }; + + using CachedValuesPtr = std::shared_ptr; + + explicit LowCardinalityDictionaryCache(const HashMethodContext::Settings & settings) : cache(settings.max_threads) {} + + CachedValuesPtr get(const DictionaryKey & key) { return cache.get(key); } + void set(const DictionaryKey & key, const CachedValuesPtr & mapped) { cache.set(key, mapped); } + +private: + using Cache = LRUCache; + Cache cache; +}; + +/// Single low cardinality column. +template +struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod +{ + using Base = SingleColumnMethod; + + static HashMethodContextPtr createContext(const HashMethodContext::Settings & settings) + { + return std::make_shared(settings); + } + + ColumnRawPtrs key_columns; + const IColumn * positions = nullptr; + size_t size_of_index_type = 0; + + /// saved hash is from current column or from cache. + const UInt64 * saved_hash = nullptr; + /// Hold dictionary in case saved_hash is from cache to be sure it won't be deleted. + ColumnPtr dictionary_holder; + + /// Cache AggregateDataPtr for current column in order to decrease the number of hash table usages. + PaddedPODArray aggregate_data_cache; + + /// If initialized column is nullable. + bool is_nullable = false; + + static const ColumnLowCardinality & getLowCardinalityColumn(const IColumn * low_cardinality_column) + { + auto column = typeid_cast(low_cardinality_column); + if (!column) + throw Exception("Invalid aggregation key type for HashMethodSingleLowCardinalityColumn method. " + "Excepted LowCardinality, got " + column->getName(), ErrorCodes::LOGICAL_ERROR); + return *column; + } + + HashMethodSingleLowCardinalityColumn( + const ColumnRawPtrs & key_columns_low_cardinality, const Sizes & key_sizes, const HashMethodContextPtr & context) + : Base({getLowCardinalityColumn(key_columns_low_cardinality[0]).getDictionary().getNestedNotNullableColumn().get()}, key_sizes, context) + { + auto column = &getLowCardinalityColumn(key_columns_low_cardinality[0]); + + if (!context) + throw Exception("Cache wasn't created for HashMethodSingleLowCardinalityColumn", + ErrorCodes::LOGICAL_ERROR); + + LowCardinalityDictionaryCache * cache; + if constexpr (use_cache) + { + cache = typeid_cast(context.get()); + if (!cache) + { + const auto & cached_val = *context; + throw Exception("Invalid type for HashMethodSingleLowCardinalityColumn cache: " + + demangle(typeid(cached_val).name()), ErrorCodes::LOGICAL_ERROR); + } + } + + auto * dict = column->getDictionary().getNestedNotNullableColumn().get(); + is_nullable = column->getDictionary().nestedColumnIsNullable(); + key_columns = {dict}; + bool is_shared_dict = column->isSharedDictionary(); + + typename LowCardinalityDictionaryCache::DictionaryKey dictionary_key; + typename LowCardinalityDictionaryCache::CachedValuesPtr cached_values; + + if (is_shared_dict) + { + dictionary_key = {column->getDictionary().getHash(), dict->size()}; + if constexpr (use_cache) + cached_values = cache->get(dictionary_key); + } + + if (cached_values) + { + saved_hash = cached_values->saved_hash; + dictionary_holder = cached_values->dictionary_holder; + } + else + { + saved_hash = column->getDictionary().tryGetSavedHash(); + dictionary_holder = column->getDictionaryPtr(); + + if constexpr (use_cache) + { + if (is_shared_dict) + { + cached_values = std::make_shared(); + cached_values->saved_hash = saved_hash; + cached_values->dictionary_holder = dictionary_holder; + + cache->set(dictionary_key, cached_values); + } + } + } + + AggregateDataPtr default_data = nullptr; + aggregate_data_cache.assign(key_columns[0]->size(), default_data); + + size_of_index_type = column->getSizeOfIndexType(); + positions = column->getIndexesPtr().get(); + } + + ALWAYS_INLINE size_t getIndexAt(size_t row) const + { + switch (size_of_index_type) + { + case sizeof(UInt8): return static_cast(positions)->getElement(row); + case sizeof(UInt16): return static_cast(positions)->getElement(row); + case sizeof(UInt32): return static_cast(positions)->getElement(row); + case sizeof(UInt64): return static_cast(positions)->getElement(row); + default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR); + } + } + + /// Get the key from the key columns for insertion into the hash table. + ALWAYS_INLINE auto getKey(size_t row) const + { + return Base::getKey(getIndexAt(row)); + } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row_, bool & inserted, Arena & pool) + { + size_t row = getIndexAt(row_); + + if (is_nullable && row == 0) + { + inserted = !data.hasNullKeyData(); + data.hasNullKeyData() = true; + return &data.getNullKeyData(); + } + + if constexpr (use_cache) + { + if (aggregate_data_cache[row]) + { + inserted = false; + return &aggregate_data_cache[row]; + } + } + + Sizes key_sizes; + auto key = getKey(row_); + + typename Data::iterator it; + if (saved_hash) + data.emplace(key, it, inserted, saved_hash[row]); + else + data.emplace(key, it, inserted); + + if (inserted) + Base::onNewKey(*it, pool); + else if constexpr (use_cache) + aggregate_data_cache[row] = it->second; + + return HashTableTraits::getMapped(*it); + } + + ALWAYS_INLINE bool isNullAt(size_t i) + { + if (!is_nullable) + return false; + + return getIndexAt(i) == 0; + } + + template + ALWAYS_INLINE void cacheData(size_t i, Mapped mapped) + { + size_t row = getIndexAt(i); + aggregate_data_cache[row] = mapped; + } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped findFromRow(Data & data, size_t row_, bool & found, Arena &) + { + size_t row = getIndexAt(row_); + + if (is_nullable && row == 0) + return data.hasNullKeyData() ? &data.getNullKeyData() : nullptr; + + if constexpr (use_cache) + { + if (aggregate_data_cache[row]) + return &aggregate_data_cache[row]; + } + + auto key = getKey(row_); + + typename Data::iterator it; + if (saved_hash) + it = data.find(key, saved_hash[row]); + else + it = data.find(key); + + found = it != data.end(); + if constexpr (use_cache) + { + if (found) + aggregate_data_cache[row] = it->second; + } + + return typename HashTableTraits::getMapped(*it); + } + + template + ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & pool) + { + row = getIndexAt(row); + if (saved_hash) + return saved_hash[row]; + + return Base::getHash(data, row, pool); + } + + template + static void insertKeyIntoColumns(const Value & value, MutableColumns & key_columns_low_cardinality, const Sizes & /*key_sizes*/) + { + auto ref = Base::getValueRef(value); + static_cast(key_columns_low_cardinality[0].get())->insertData(ref.data, ref.size); + } +}; + + +namespace columns_hashing_impl +{ + +/// This class is designed to provide the functionality that is required for +/// supporting nullable keys in HashMethodKeysFixed. If there are +/// no nullable keys, this class is merely implemented as an empty shell. +template +class BaseStateKeysFixed; + +/// Case where nullable keys are supported. +template +class BaseStateKeysFixed +{ +protected: + void init(const ColumnRawPtrs & key_columns) + { + null_maps.reserve(key_columns.size()); + actual_columns.reserve(key_columns.size()); + + for (const auto & col : key_columns) + { + if (col->isColumnNullable()) + { + const auto & nullable_col = static_cast(*col); + actual_columns.push_back(&nullable_col.getNestedColumn()); + null_maps.push_back(&nullable_col.getNullMapColumn()); + } + else + { + actual_columns.push_back(col); + null_maps.push_back(nullptr); + } + } + } + + /// Return the columns which actually contain the values of the keys. + /// For a given key column, if it is nullable, we return its nested + /// column. Otherwise we return the key column itself. + inline const ColumnRawPtrs & getActualColumns() const + { + return actual_columns; + } + + /// Create a bitmap that indicates whether, for a particular row, + /// a key column bears a null value or not. + KeysNullMap createBitmap(size_t row) const + { + KeysNullMap bitmap{}; + + for (size_t k = 0; k < null_maps.size(); ++k) + { + if (null_maps[k] != nullptr) + { + const auto & null_map = static_cast(*null_maps[k]).getData(); + if (null_map[row] == 1) + { + size_t bucket = k / 8; + size_t offset = k % 8; + bitmap[bucket] |= UInt8(1) << offset; + } + } + } + + return bitmap; + } + +private: + ColumnRawPtrs actual_columns; + ColumnRawPtrs null_maps; +}; + +/// Case where nullable keys are not supported. +template +class BaseStateKeysFixed +{ +protected: + void init(const ColumnRawPtrs & columns) { actual_columns = columns; } + + const ColumnRawPtrs & getActualColumns() const { return actual_columns; } + + KeysNullMap createBitmap(size_t) const + { + throw Exception{"Internal error: calling createBitmap() for non-nullable keys" + " is forbidden", ErrorCodes::LOGICAL_ERROR}; + } + +private: + ColumnRawPtrs actual_columns; +}; + +} + +// Optional mask for low cardinality columns. +template +struct LowCardinalityKeys +{ + ColumnRawPtrs nested_columns; + ColumnRawPtrs positions; + Sizes position_sizes; +}; + +template <> +struct LowCardinalityKeys {}; + +/// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits. +template +struct HashMethodKeysFixed : private columns_hashing_impl::BaseStateKeysFixed +{ + using Key = typename TData::key_type; + + static constexpr bool has_nullable_keys = has_nullable_keys_; + static constexpr bool has_low_cardinality = has_low_cardinality_; + + LowCardinalityKeys low_cardinality_keys; + Sizes key_sizes; + size_t keys_size; + + LastElementCache last_elem_cache; + + using Base = columns_hashing_impl::BaseStateKeysFixed; + + HashMethodKeysFixed(const ColumnRawPtrs & key_columns, const Sizes & key_sizes, const HashMethodContextPtr &) + : key_sizes(std::move(key_sizes)), keys_size(key_columns.size()) + { + if constexpr (has_low_cardinality) + { + low_cardinality_keys.nested_columns.resize(key_columns.size()); + low_cardinality_keys.positions.assign(key_columns.size(), nullptr); + low_cardinality_keys.position_sizes.resize(key_columns.size()); + for (size_t i = 0; i < key_columns.size(); ++i) + { + if (auto * low_cardinality_col = typeid_cast(key_columns[i])) + { + low_cardinality_keys.nested_columns[i] = low_cardinality_col->getDictionary().getNestedColumn().get(); + low_cardinality_keys.positions[i] = &low_cardinality_col->getIndexes(); + low_cardinality_keys.position_sizes[i] = low_cardinality_col->getSizeOfIndexType(); + } + else + low_cardinality_keys.nested_columns[i] = key_columns[i]; + } + } + + Base::init(key_columns); + } + + static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; } + + ALWAYS_INLINE Key getKey(size_t row) const + { + if (has_nullable_keys) + { + auto bitmap = Base::createBitmap(row); + return packFixed(row, keys_size, Base::getActualColumns(), key_sizes, bitmap); + } + else + { + if constexpr (has_low_cardinality) + return packFixed(row, keys_size, low_cardinality_keys.nested_columns, key_sizes, + &low_cardinality_keys.positions, &low_cardinality_keys.position_sizes); + + return packFixed(row, keys_size, Base::getActualColumns(), key_sizes); + } + } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row, bool & inserted, Arena & /*pool*/) + { + return HashTableTraits::getMapped(emplaceKeyImpl(getKey(row), data, inserted, last_elem_cache)); + } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & /*pool*/) + { + return findKeyImpl(getKey(row), data, found, last_elem_cache); + } + + template + static StringRef getValueRef(const Value & value) + { + return StringRef(value.first.data, value.first.size); + } + + template + ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & /*pool*/) + { + return data.hash(getKey(row)); + } + + template + ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped mapped) + { + *last_elem_cache.getMapped() = mapped; + } +}; + +/** Hash by concatenating serialized key values. + * The serialized value differs in that it uniquely allows to deserialize it, having only the position with which it starts. + * That is, for example, for strings, it contains first the serialized length of the string, and then the bytes. + * Therefore, when aggregating by several strings, there is no ambiguity. + */ +template +struct HashMethodSerialized +{ + ColumnRawPtrs key_columns; + size_t keys_size; + LastElementCache last_elem_cache; + + HashMethodSerialized(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) + : key_columns(key_columns), keys_size(key_columns.size()) {} + + static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row, bool & inserted, Arena & pool) + { + auto key = getKey(row, pool); + auto & value = emplaceKeyImpl(key, data, inserted, last_elem_cache); + if (!inserted) + pool.rollback(key.size); + + return HashTableTraits::getMapped(value); + } + + template + ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & pool) + { + auto key = getKey(row, pool); + auto mapped = findKeyImpl(key, data, found, last_elem_cache); + pool.rollback(key.size); + + return mapped; + } + + template + ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & pool) + { + auto key = getKey(row, pool); + auto hash = data.hash(key); + pool.rollback(key.size); + + return hash; + } + + template + ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped /*mapped*/) {} + +protected: + ALWAYS_INLINE StringRef getKey(size_t row, Arena & pool) const + { + return serializeKeysToPoolContiguous(row, keys_size, key_columns, pool); + } +}; + +} +} diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 145ce98dbbc..785345f9400 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -6,9 +6,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -22,11 +24,9 @@ #include #include #include + #if __has_include() #include -#include -#include - #endif @@ -188,7 +188,7 @@ Aggregator::Aggregator(const Params & params_) } method_chosen = chooseAggregationMethod(); - AggregationStateCache::Settings cache_settings; + HashMethodContext::Settings cache_settings; cache_settings.max_threads = params.max_threads; aggregation_state_cache = AggregatedDataVariants::createCache(method_chosen, cache_settings); } @@ -586,11 +586,7 @@ void NO_INLINE Aggregator::executeImpl( bool no_more_keys, AggregateDataPtr overflow_row) const { - typename Method::State state; - if constexpr (Method::low_cardinality_optimization) - state.init(key_columns, aggregation_state_cache); - else - state.init(key_columns); + typename Method::State state(key_columns, key_sizes, aggregation_state_cache); if (!no_more_keys) executeImplCase(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, keys, overflow_row); @@ -605,76 +601,35 @@ void NO_INLINE Aggregator::executeImplCase( typename Method::State & state, Arena * aggregates_pool, size_t rows, - ColumnRawPtrs & key_columns, + ColumnRawPtrs & /*key_columns*/, AggregateFunctionInstruction * aggregate_instructions, - StringRefs & keys, + StringRefs & /*keys*/, AggregateDataPtr overflow_row) const { /// NOTE When editing this code, also pay attention to SpecializedAggregator.h. /// For all rows. - typename Method::Key prev_key{}; AggregateDataPtr value = nullptr; for (size_t i = 0; i < rows; ++i) { bool inserted = false; /// Inserted a new key, or was this key already? - /// Get the key to insert into the hash table. - typename Method::Key key; - if constexpr (!Method::low_cardinality_optimization) - key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool); - AggregateDataPtr * aggregate_data = nullptr; - typename Method::iterator it; /// Is not used if Method::low_cardinality_optimization - if (!no_more_keys) /// Insert. - { - /// Optimization for consecutive identical keys. - if (!Method::no_consecutive_keys_optimization) - { - if (i != 0 && key == prev_key) - { - /// Add values to the aggregate functions. - for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) - (*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i, aggregates_pool); - - method.onExistingKey(key, keys, *aggregates_pool); - continue; - } - else - prev_key = key; - } - - if constexpr (Method::low_cardinality_optimization) - aggregate_data = state.emplaceKeyFromRow(method.data, i, inserted, params.keys_size, keys, *aggregates_pool); - else - { - method.data.emplace(key, it, inserted); - aggregate_data = &Method::getAggregateData(it->second); - } - } + if constexpr (!no_more_keys) /// Insert. + aggregate_data = state.emplaceKey(method.data, i, inserted, *aggregates_pool); else { /// Add only if the key already exists. - - if constexpr (Method::low_cardinality_optimization) - aggregate_data = state.findFromRow(method.data, i); - else - { - it = method.data.find(key); - if (method.data.end() != it) - aggregate_data = &Method::getAggregateData(it->second); - } + bool found = false; + aggregate_data = state.findKey(method.data, i, found, *aggregates_pool); } /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys. /// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do. if (!aggregate_data && !overflow_row) - { - method.onExistingKey(key, keys, *aggregates_pool); continue; - } /// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. if (inserted) @@ -682,18 +637,12 @@ void NO_INLINE Aggregator::executeImplCase( /// exception-safety - if you can not allocate memory or create states, then destructors will not be called. *aggregate_data = nullptr; - if constexpr (!Method::low_cardinality_optimization) - method.onNewKey(*it, params.keys_size, keys, *aggregates_pool); - AggregateDataPtr place = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); createAggregateStates(place); *aggregate_data = place; - if constexpr (Method::low_cardinality_optimization) - state.cacheAggregateData(i, place); + state.cacheData(i, place); } - else - method.onExistingKey(key, keys, *aggregates_pool); value = aggregate_data ? *aggregate_data : overflow_row; @@ -1172,7 +1121,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal( for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_functions[i]->insertResultInto( - Method::getAggregateData(value.second) + offsets_of_aggregate_states[i], + value.second + offsets_of_aggregate_states[i], *final_aggregate_columns[i]); } @@ -1203,9 +1152,9 @@ void NO_INLINE Aggregator::convertToBlockImplNotFinal( /// reserved, so push_back does not throw exceptions for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i]->push_back(Method::getAggregateData(value.second) + offsets_of_aggregate_states[i]); + aggregate_columns[i]->push_back(value.second + offsets_of_aggregate_states[i]); - Method::getAggregateData(value.second) = nullptr; + value.second = nullptr; } } @@ -1549,20 +1498,20 @@ void NO_INLINE Aggregator::mergeDataImpl( { for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_functions[i]->merge( - Method::getAggregateData(res_it->second) + offsets_of_aggregate_states[i], - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i], + res_it->second + offsets_of_aggregate_states[i], + it->second + offsets_of_aggregate_states[i], arena); for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_functions[i]->destroy( - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + it->second + offsets_of_aggregate_states[i]); } else { res_it->second = it->second; } - Method::getAggregateData(it->second) = nullptr; + it->second = nullptr; } table_src.clearAndShrink(); @@ -1586,19 +1535,18 @@ void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl( AggregateDataPtr res_data = table_dst.end() == res_it ? overflows - : Method::getAggregateData(res_it->second); + : res_it->second; for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_functions[i]->merge( res_data + offsets_of_aggregate_states[i], - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i], + it->second + offsets_of_aggregate_states[i], arena); for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_functions[i]->destroy( - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + aggregate_functions[i]->destroy(it->second + offsets_of_aggregate_states[i]); - Method::getAggregateData(it->second) = nullptr; + it->second = nullptr; } table_src.clearAndShrink(); @@ -1621,19 +1569,18 @@ void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl( if (table_dst.end() == res_it) continue; - AggregateDataPtr res_data = Method::getAggregateData(res_it->second); + AggregateDataPtr res_data = res_it->second; for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_functions[i]->merge( res_data + offsets_of_aggregate_states[i], - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i], + it->second + offsets_of_aggregate_states[i], arena); for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_functions[i]->destroy( - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + aggregate_functions[i]->destroy(it->second + offsets_of_aggregate_states[i]); - Method::getAggregateData(it->second) = nullptr; + it->second = nullptr; } table_src.clearAndShrink(); @@ -1984,7 +1931,7 @@ template void NO_INLINE Aggregator::mergeStreamsImplCase( Block & block, Arena * aggregates_pool, - Method & method, + Method & method [[maybe_unused]], Table & data, AggregateDataPtr overflow_row) const { @@ -1998,14 +1945,9 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_columns[i] = &typeid_cast(*block.safeGetByPosition(params.keys_size + i).column).getData(); - typename Method::State state; - if constexpr (Method::low_cardinality_optimization) - state.init(key_columns, aggregation_state_cache); - else - state.init(key_columns); + typename Method::State state(key_columns, key_sizes, aggregation_state_cache); /// For all rows. - StringRefs keys(params.keys_size); size_t rows = block.rows(); for (size_t i = 0; i < rows; ++i) { @@ -2014,59 +1956,31 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( bool inserted = false; /// Inserted a new key, or was this key already? - /// Get the key to insert into the hash table. - typename Method::Key key; - if constexpr (!Method::low_cardinality_optimization) - key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool); - if (!no_more_keys) - { - if constexpr (Method::low_cardinality_optimization) - aggregate_data = state.emplaceKeyFromRow(data, i, inserted, params.keys_size, keys, *aggregates_pool); - else - { - data.emplace(key, it, inserted); - aggregate_data = &Method::getAggregateData(it->second); - } - } + aggregate_data = state.emplaceKey(data, i, inserted, *aggregates_pool); else { - if constexpr (Method::low_cardinality_optimization) - aggregate_data = state.findFromRow(data, i); - else - { - it = data.find(key); - if (data.end() != it) - aggregate_data = &Method::getAggregateData(it->second); - } + bool found; + aggregate_data = state.findKey(data, i, found, *aggregates_pool); } /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys. /// If the key does not fit, and the data does not need to be aggregated into a separate row, then there's nothing to do. if (!aggregate_data && !overflow_row) - { - method.onExistingKey(key, keys, *aggregates_pool); continue; - } /// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. if (inserted) { *aggregate_data = nullptr; - if constexpr (!Method::low_cardinality_optimization) - method.onNewKey(*it, params.keys_size, keys, *aggregates_pool); - AggregateDataPtr place = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); createAggregateStates(place); *aggregate_data = place; - if constexpr (Method::low_cardinality_optimization) - state.cacheAggregateData(i, place); + state.cacheData(i, place); } - else - method.onExistingKey(key, keys, *aggregates_pool); AggregateDataPtr value = aggregate_data ? *aggregate_data : overflow_row; @@ -2163,7 +2077,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV * If there is at least one block with a bucket number greater or equal than zero, then there was a two-level aggregation. */ auto max_bucket = bucket_to_blocks.rbegin()->first; - size_t has_two_level = max_bucket >= 0; + bool has_two_level = max_bucket >= 0; if (has_two_level) { @@ -2393,15 +2307,11 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl( Method & method, Arena * pool, ColumnRawPtrs & key_columns, - StringRefs & keys, + StringRefs & keys [[maybe_unused]], const Block & source, std::vector & destinations) const { - typename Method::State state; - if constexpr (Method::low_cardinality_optimization) - state.init(key_columns, aggregation_state_cache); - else - state.init(key_columns); + typename Method::State state(key_columns, key_sizes, aggregation_state_cache); size_t rows = source.rows(); size_t columns = source.columns(); @@ -2421,16 +2331,11 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl( } } - /// Obtain a key. Calculate bucket number from it. - typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *pool); - - auto hash = method.data.hash(key); + /// Calculate bucket number from row hash. + auto hash = state.getHash(method.data, i, *pool); auto bucket = method.data.getBucketFromHash(hash); selector[i] = bucket; - - /// We don't need to store this key in pool. - method.onExistingKey(key, keys, *pool); } size_t num_buckets = destinations.size(); @@ -2521,7 +2426,7 @@ void NO_INLINE Aggregator::destroyImpl(Table & table) const { for (auto elem : table) { - AggregateDataPtr & data = Method::getAggregateData(elem.second); + AggregateDataPtr & data = elem.second; /** If an exception (usually a lack of memory, the MemoryTracker throws) arose * after inserting the key into a hash table, but before creating all states of aggregate functions, diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index f51f620064f..0b40f4e6a25 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -138,18 +139,6 @@ using AggregatedDataWithNullableStringKeyTwoLevel = AggregationDataWithNullKeyTw TwoLevelHashMapWithSavedHash, TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>; -/// Cache which can be used by aggregations method's states. Object is shared in all threads. -struct AggregationStateCache -{ - virtual ~AggregationStateCache() = default; - - struct Settings - { - size_t max_threads; - }; -}; - -using AggregationStateCachePtr = std::shared_ptr; /// For the case where there is one numeric key. template /// UInt8/16/32/64 for any type with corresponding bit width. @@ -169,65 +158,16 @@ struct AggregationMethodOneNumber AggregationMethodOneNumber(const Other & other) : data(other.data) {} /// To use one `Method` in different threads, use different `State`. - struct State - { - const char * vec; + using State = ColumnsHashing::HashMethodOneNumber; - /** Called at the start of each block processing. - * Sets the variables needed for the other methods called in inner loops. - */ - void init(ColumnRawPtrs & key_columns) - { - vec = key_columns[0]->getRawData().data; - } - - /// Get the key from the key columns for insertion into the hash table. - ALWAYS_INLINE Key getKey( - const ColumnRawPtrs & /*key_columns*/, - size_t /*keys_size*/, /// Number of key columns. - size_t i, /// From which row of the block, get the key. - const Sizes & /*key_sizes*/, /// If the keys of a fixed length - their lengths. It is not used in aggregation methods for variable length keys. - StringRefs & /*keys*/, /// Here references to key data in columns can be written. They can be used in the future. - Arena & /*pool*/) const - { - return unalignedLoad(vec + i * sizeof(FieldType)); - } - }; - - /// From the value in the hash table, get AggregateDataPtr. - static AggregateDataPtr & getAggregateData(Mapped & value) { return value; } - static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; } - - /** Place additional data, if necessary, in case a new key was inserted into the hash table. - */ - static ALWAYS_INLINE void onNewKey(typename Data::value_type & /*value*/, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & /*pool*/) - { - } - - /** The action to be taken if the key is not new. For example, roll back the memory allocation in the pool. - */ - static ALWAYS_INLINE void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {} - - /** Do not use optimization for consecutive keys. - */ - static const bool no_consecutive_keys_optimization = false; /// Use optimization for low cardinality. static const bool low_cardinality_optimization = false; - /** Insert the key from the hash table into columns. - */ + // Insert the key from the hash table into columns. static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) { static_cast(key_columns[0].get())->insertRawData(reinterpret_cast(&value.first)); } - - /// Get StringRef from value which can be inserted into column. - static StringRef getValueRef(const typename Data::value_type & value) - { - return StringRef(reinterpret_cast(&value.first), sizeof(value.first)); - } - - static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; } }; @@ -248,58 +188,14 @@ struct AggregationMethodString template AggregationMethodString(const Other & other) : data(other.data) {} - struct State - { - const IColumn::Offset * offsets; - const UInt8 * chars; + using State = ColumnsHashing::HashMethodString; - void init(ColumnRawPtrs & key_columns) - { - const IColumn & column = *key_columns[0]; - const ColumnString & column_string = static_cast(column); - offsets = column_string.getOffsets().data(); - chars = column_string.getChars().data(); - } - - ALWAYS_INLINE Key getKey( - const ColumnRawPtrs & /*key_columns*/, - size_t /*keys_size*/, - ssize_t i, - const Sizes & /*key_sizes*/, - StringRefs & /*keys*/, - Arena & /*pool*/) const - { - return StringRef( - chars + offsets[i - 1], - offsets[i] - offsets[i - 1] - 1); - } - }; - - static AggregateDataPtr & getAggregateData(Mapped & value) { return value; } - static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; } - - static ALWAYS_INLINE void onNewKey(typename Data::value_type & value, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & pool) - { - if (value.first.size) - value.first.data = pool.insert(value.first.data, value.first.size); - } - - static ALWAYS_INLINE void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {} - - static const bool no_consecutive_keys_optimization = false; static const bool low_cardinality_optimization = false; - static StringRef getValueRef(const typename Data::value_type & value) - { - return StringRef(value.first.data, value.first.size); - } - static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) { key_columns[0]->insertData(value.first.data, value.first.size); } - - static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; } }; @@ -320,101 +216,14 @@ struct AggregationMethodFixedString template AggregationMethodFixedString(const Other & other) : data(other.data) {} - struct State - { - size_t n; - const ColumnFixedString::Chars * chars; + using State = ColumnsHashing::HashMethodFixedString; - void init(ColumnRawPtrs & key_columns) - { - const IColumn & column = *key_columns[0]; - const ColumnFixedString & column_string = static_cast(column); - n = column_string.getN(); - chars = &column_string.getChars(); - } - - ALWAYS_INLINE Key getKey( - const ColumnRawPtrs &, - size_t, - size_t i, - const Sizes &, - StringRefs &, - Arena &) const - { - return StringRef(&(*chars)[i * n], n); - } - }; - - static AggregateDataPtr & getAggregateData(Mapped & value) { return value; } - static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; } - - static ALWAYS_INLINE void onNewKey(typename Data::value_type & value, size_t, StringRefs &, Arena & pool) - { - value.first.data = pool.insert(value.first.data, value.first.size); - } - - static ALWAYS_INLINE void onExistingKey(const Key &, StringRefs &, Arena &) {} - - static const bool no_consecutive_keys_optimization = false; static const bool low_cardinality_optimization = false; - static StringRef getValueRef(const typename Data::value_type & value) - { - return StringRef(value.first.data, value.first.size); - } - static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) { key_columns[0]->insertData(value.first.data, value.first.size); } - - static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; } -}; - -/// Cache stores dictionaries and saved_hash per dictionary key. -class LowCardinalityDictionaryCache : public AggregationStateCache -{ -public: - /// Will assume that dictionaries with same hash has the same keys. - /// Just in case, check that they have also the same size. - struct DictionaryKey - { - UInt128 hash; - UInt64 size; - - bool operator== (const DictionaryKey & other) const { return hash == other.hash && size == other.size; } - }; - - struct DictionaryKeyHash - { - size_t operator()(const DictionaryKey & key) const - { - SipHash hash; - hash.update(key.hash.low); - hash.update(key.hash.high); - hash.update(key.size); - return hash.get64(); - } - }; - - struct CachedValues - { - /// Store ptr to dictionary to be sure it won't be deleted. - ColumnPtr dictionary_holder; - /// Hashes for dictionary keys. - const UInt64 * saved_hash = nullptr; - }; - - using CachedValuesPtr = std::shared_ptr; - - explicit LowCardinalityDictionaryCache(const AggregationStateCache::Settings & settings) : cache(settings.max_threads) {} - - CachedValuesPtr get(const DictionaryKey & key) { return cache.get(key); } - void set(const DictionaryKey & key, const CachedValuesPtr & mapped) { cache.set(key, mapped); } - -private: - using Cache = LRUCache; - Cache cache; }; /// Single low cardinality column. @@ -432,342 +241,23 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod using Base::data; - static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & settings) - { - return std::make_shared(settings); - } - AggregationMethodSingleLowCardinalityColumn() = default; template explicit AggregationMethodSingleLowCardinalityColumn(const Other & other) : Base(other) {} - struct State : public BaseState - { - ColumnRawPtrs key_columns; - const IColumn * positions = nullptr; - size_t size_of_index_type = 0; + using State = ColumnsHashing::HashMethodSingleLowCardinalityColumn; - /// saved hash is from current column or from cache. - const UInt64 * saved_hash = nullptr; - /// Hold dictionary in case saved_hash is from cache to be sure it won't be deleted. - ColumnPtr dictionary_holder; - - /// Cache AggregateDataPtr for current column in order to decrease the number of hash table usages. - PaddedPODArray aggregate_data_cache; - - /// If initialized column is nullable. - bool is_nullable = false; - - void init(ColumnRawPtrs &) - { - throw Exception("Expected cache for AggregationMethodSingleLowCardinalityColumn::init", ErrorCodes::LOGICAL_ERROR); - } - - void init(ColumnRawPtrs & key_columns_low_cardinality, const AggregationStateCachePtr & cache_ptr) - { - auto column = typeid_cast(key_columns_low_cardinality[0]); - if (!column) - throw Exception("Invalid aggregation key type for AggregationMethodSingleLowCardinalityColumn method. " - "Excepted LowCardinality, got " + key_columns_low_cardinality[0]->getName(), ErrorCodes::LOGICAL_ERROR); - - if (!cache_ptr) - throw Exception("Cache wasn't created for AggregationMethodSingleLowCardinalityColumn", ErrorCodes::LOGICAL_ERROR); - - auto cache = typeid_cast(cache_ptr.get()); - if (!cache) - { - const auto & cached_val = *cache_ptr; - throw Exception("Invalid type for AggregationMethodSingleLowCardinalityColumn cache: " - + demangle(typeid(cached_val).name()), ErrorCodes::LOGICAL_ERROR); - } - - auto * dict = column->getDictionary().getNestedNotNullableColumn().get(); - is_nullable = column->getDictionary().nestedColumnIsNullable(); - key_columns = {dict}; - bool is_shared_dict = column->isSharedDictionary(); - - typename LowCardinalityDictionaryCache::DictionaryKey dictionary_key; - typename LowCardinalityDictionaryCache::CachedValuesPtr cached_values; - - if (is_shared_dict) - { - dictionary_key = {column->getDictionary().getHash(), dict->size()}; - cached_values = cache->get(dictionary_key); - } - - if (cached_values) - { - saved_hash = cached_values->saved_hash; - dictionary_holder = cached_values->dictionary_holder; - } - else - { - saved_hash = column->getDictionary().tryGetSavedHash(); - dictionary_holder = column->getDictionaryPtr(); - - if (is_shared_dict) - { - cached_values = std::make_shared(); - cached_values->saved_hash = saved_hash; - cached_values->dictionary_holder = dictionary_holder; - - cache->set(dictionary_key, cached_values); - } - } - - AggregateDataPtr default_data = nullptr; - aggregate_data_cache.assign(key_columns[0]->size(), default_data); - - size_of_index_type = column->getSizeOfIndexType(); - positions = column->getIndexesPtr().get(); - - BaseState::init(key_columns); - } - - ALWAYS_INLINE size_t getIndexAt(size_t row) const - { - switch (size_of_index_type) - { - case sizeof(UInt8): return static_cast(positions)->getElement(row); - case sizeof(UInt16): return static_cast(positions)->getElement(row); - case sizeof(UInt32): return static_cast(positions)->getElement(row); - case sizeof(UInt64): return static_cast(positions)->getElement(row); - default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR); - } - } - - /// Get the key from the key columns for insertion into the hash table. - ALWAYS_INLINE Key getKey( - const ColumnRawPtrs & /*key_columns*/, - size_t /*keys_size*/, - size_t i, - const Sizes & key_sizes, - StringRefs & keys, - Arena & pool) const - { - size_t row = getIndexAt(i); - return BaseState::getKey(key_columns, 1, row, key_sizes, keys, pool); - } - - template - ALWAYS_INLINE AggregateDataPtr * emplaceKeyFromRow( - D & data, - size_t i, - bool & inserted, - size_t keys_size, - StringRefs & keys, - Arena & pool) - { - size_t row = getIndexAt(i); - - if (is_nullable && row == 0) - { - inserted = !data.hasNullKeyData(); - data.hasNullKeyData() = true; - return &data.getNullKeyData(); - } - - if (aggregate_data_cache[row]) - { - inserted = false; - return &aggregate_data_cache[row]; - } - else - { - Sizes key_sizes; - auto key = getKey({}, 0, i, key_sizes, keys, pool); - - typename D::iterator it; - if (saved_hash) - data.emplace(key, it, inserted, saved_hash[row]); - else - data.emplace(key, it, inserted); - - if (inserted) - Base::onNewKey(*it, keys_size, keys, pool); - else - aggregate_data_cache[row] = Base::getAggregateData(it->second); - - return &Base::getAggregateData(it->second); - } - } - - ALWAYS_INLINE bool isNullAt(size_t i) - { - if (!is_nullable) - return false; - - return getIndexAt(i) == 0; - } - - ALWAYS_INLINE void cacheAggregateData(size_t i, AggregateDataPtr data) - { - size_t row = getIndexAt(i); - aggregate_data_cache[row] = data; - } - - template - ALWAYS_INLINE AggregateDataPtr * findFromRow(D & data, size_t i) - { - size_t row = getIndexAt(i); - - if (is_nullable && row == 0) - return data.hasNullKeyData() ? &data.getNullKeyData() : nullptr; - - if (!aggregate_data_cache[row]) - { - Sizes key_sizes; - StringRefs keys; - Arena pool; - auto key = getKey({}, 0, i, key_sizes, keys, pool); - - typename D::iterator it; - if (saved_hash) - it = data.find(key, saved_hash[row]); - else - it = data.find(key); - - if (it != data.end()) - aggregate_data_cache[row] = Base::getAggregateData(it->second); - } - return &aggregate_data_cache[row]; - } - }; - - static AggregateDataPtr & getAggregateData(Mapped & value) { return Base::getAggregateData(value); } - static const AggregateDataPtr & getAggregateData(const Mapped & value) { return Base::getAggregateData(value); } - - static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool) - { - return Base::onNewKey(value, keys_size, keys, pool); - } - - static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) - { - return Base::onExistingKey(key, keys, pool); - } - - static const bool no_consecutive_keys_optimization = true; static const bool low_cardinality_optimization = true; static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns_low_cardinality, const Sizes & /*key_sizes*/) { - auto ref = Base::getValueRef(value); + auto ref = BaseState::getValueRef(value); static_cast(key_columns_low_cardinality[0].get())->insertData(ref.data, ref.size); } }; -namespace aggregator_impl -{ - -/// This class is designed to provide the functionality that is required for -/// supporting nullable keys in AggregationMethodKeysFixed. If there are -/// no nullable keys, this class is merely implemented as an empty shell. -template -class BaseStateKeysFixed; - -/// Case where nullable keys are supported. -template -class BaseStateKeysFixed -{ -protected: - void init(const ColumnRawPtrs & key_columns) - { - null_maps.reserve(key_columns.size()); - actual_columns.reserve(key_columns.size()); - - for (const auto & col : key_columns) - { - if (col->isColumnNullable()) - { - const auto & nullable_col = static_cast(*col); - actual_columns.push_back(&nullable_col.getNestedColumn()); - null_maps.push_back(&nullable_col.getNullMapColumn()); - } - else - { - actual_columns.push_back(col); - null_maps.push_back(nullptr); - } - } - } - - /// Return the columns which actually contain the values of the keys. - /// For a given key column, if it is nullable, we return its nested - /// column. Otherwise we return the key column itself. - inline const ColumnRawPtrs & getActualColumns() const - { - return actual_columns; - } - - /// Create a bitmap that indicates whether, for a particular row, - /// a key column bears a null value or not. - KeysNullMap createBitmap(size_t row) const - { - KeysNullMap bitmap{}; - - for (size_t k = 0; k < null_maps.size(); ++k) - { - if (null_maps[k] != nullptr) - { - const auto & null_map = static_cast(*null_maps[k]).getData(); - if (null_map[row] == 1) - { - size_t bucket = k / 8; - size_t offset = k % 8; - bitmap[bucket] |= UInt8(1) << offset; - } - } - } - - return bitmap; - } - -private: - ColumnRawPtrs actual_columns; - ColumnRawPtrs null_maps; -}; - -/// Case where nullable keys are not supported. -template -class BaseStateKeysFixed -{ -protected: - void init(const ColumnRawPtrs &) - { - throw Exception{"Internal error: calling init() for non-nullable" - " keys is forbidden", ErrorCodes::LOGICAL_ERROR}; - } - - const ColumnRawPtrs & getActualColumns() const - { - throw Exception{"Internal error: calling getActualColumns() for non-nullable" - " keys is forbidden", ErrorCodes::LOGICAL_ERROR}; - } - - KeysNullMap createBitmap(size_t) const - { - throw Exception{"Internal error: calling createBitmap() for non-nullable keys" - " is forbidden", ErrorCodes::LOGICAL_ERROR}; - } -}; - -} - -// Oprional mask for low cardinality columns. -template -struct LowCardinalityKeys -{ - ColumnRawPtrs nested_columns; - ColumnRawPtrs positions; - Sizes position_sizes; -}; - -template <> -struct LowCardinalityKeys {}; - /// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits. template struct AggregationMethodKeysFixed @@ -787,71 +277,8 @@ struct AggregationMethodKeysFixed template AggregationMethodKeysFixed(const Other & other) : data(other.data) {} - class State final : private aggregator_impl::BaseStateKeysFixed - { - LowCardinalityKeys low_cardinality_keys; + using State = ColumnsHashing::HashMethodKeysFixed; - public: - using Base = aggregator_impl::BaseStateKeysFixed; - - void init(ColumnRawPtrs & key_columns) - { - if constexpr (has_low_cardinality) - { - low_cardinality_keys.nested_columns.resize(key_columns.size()); - low_cardinality_keys.positions.assign(key_columns.size(), nullptr); - low_cardinality_keys.position_sizes.resize(key_columns.size()); - for (size_t i = 0; i < key_columns.size(); ++i) - { - if (auto * low_cardinality_col = typeid_cast(key_columns[i])) - { - low_cardinality_keys.nested_columns[i] = low_cardinality_col->getDictionary().getNestedColumn().get(); - low_cardinality_keys.positions[i] = &low_cardinality_col->getIndexes(); - low_cardinality_keys.position_sizes[i] = low_cardinality_col->getSizeOfIndexType(); - } - else - low_cardinality_keys.nested_columns[i] = key_columns[i]; - } - } - - if (has_nullable_keys) - Base::init(key_columns); - } - - ALWAYS_INLINE Key getKey( - const ColumnRawPtrs & key_columns, - size_t keys_size, - size_t i, - const Sizes & key_sizes, - StringRefs &, - Arena &) const - { - if (has_nullable_keys) - { - auto bitmap = Base::createBitmap(i); - return packFixed(i, keys_size, Base::getActualColumns(), key_sizes, bitmap); - } - else - { - if constexpr (has_low_cardinality) - return packFixed(i, keys_size, low_cardinality_keys.nested_columns, key_sizes, - &low_cardinality_keys.positions, &low_cardinality_keys.position_sizes); - - return packFixed(i, keys_size, key_columns, key_sizes); - } - } - }; - - static AggregateDataPtr & getAggregateData(Mapped & value) { return value; } - static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; } - - static ALWAYS_INLINE void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &) - { - } - - static ALWAYS_INLINE void onExistingKey(const Key &, StringRefs &, Arena &) {} - - static const bool no_consecutive_keys_optimization = false; static const bool low_cardinality_optimization = false; static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & key_sizes) @@ -904,8 +331,6 @@ struct AggregationMethodKeysFixed } } } - - static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; } }; @@ -930,53 +355,24 @@ struct AggregationMethodSerialized template AggregationMethodSerialized(const Other & other) : data(other.data) {} - struct State - { - void init(ColumnRawPtrs &) - { - } + using State = ColumnsHashing::HashMethodSerialized; - ALWAYS_INLINE Key getKey( - const ColumnRawPtrs & key_columns, - size_t keys_size, - size_t i, - const Sizes &, - StringRefs &, - Arena & pool) const - { - return serializeKeysToPoolContiguous(i, keys_size, key_columns, pool); - } - }; - - static AggregateDataPtr & getAggregateData(Mapped & value) { return value; } - static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; } - - static ALWAYS_INLINE void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &) - { - } - - static ALWAYS_INLINE void onExistingKey(const Key & key, StringRefs &, Arena & pool) - { - pool.rollback(key.size); - } - - /// If the key already was, it is removed from the pool (overwritten), and the next key can not be compared with it. - static const bool no_consecutive_keys_optimization = true; static const bool low_cardinality_optimization = false; static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) { auto pos = value.first.data; - for (size_t i = 0; i < key_columns.size(); ++i) - pos = key_columns[i]->deserializeAndInsertFromArena(pos); + for (auto & column : key_columns) + pos = column->deserializeAndInsertFromArena(pos); } - - static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; } }; class Aggregator; +using ColumnsHashing::HashMethodContext; +using ColumnsHashing::HashMethodContextPtr; + struct AggregatedDataVariants : private boost::noncopyable { /** Working with states of aggregate functions in the pool is arranged in the following (inconvenient) way: @@ -1298,7 +694,7 @@ struct AggregatedDataVariants : private boost::noncopyable } } - static AggregationStateCachePtr createCache(Type type, const AggregationStateCache::Settings & settings) + static HashMethodContextPtr createCache(Type type, const HashMethodContext::Settings & settings) { switch (type) { @@ -1309,7 +705,7 @@ struct AggregatedDataVariants : private boost::noncopyable { \ using TPtr ## NAME = decltype(AggregatedDataVariants::NAME); \ using T ## NAME = typename TPtr ## NAME ::element_type; \ - return T ## NAME ::createCache(settings); \ + return T ## NAME ::State::createContext(settings); \ } APPLY_FOR_AGGREGATED_VARIANTS(M) @@ -1496,7 +892,7 @@ protected: AggregatedDataVariants::Type method_chosen; Sizes key_sizes; - AggregationStateCachePtr aggregation_state_cache; + HashMethodContextPtr aggregation_state_cache; AggregateFunctionsPlainPtrs aggregate_functions; From 6fce028b56eb97ccfd65f3ce9624d28dfcc603fb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 Jan 2019 13:39:53 +0300 Subject: [PATCH 017/324] Refactor ColumnsHashing. --- dbms/src/Common/ColumnsHashing.h | 484 +++++++-------------------- dbms/src/Common/ColumnsHashingImpl.h | 276 +++++++++++++++ dbms/src/Interpreters/Aggregator.cpp | 82 ++--- dbms/src/Interpreters/Aggregator.h | 12 +- 4 files changed, 447 insertions(+), 407 deletions(-) create mode 100644 dbms/src/Common/ColumnsHashingImpl.h diff --git a/dbms/src/Common/ColumnsHashing.h b/dbms/src/Common/ColumnsHashing.h index 0a6d5464341..61c3d71cbbc 100644 --- a/dbms/src/Common/ColumnsHashing.h +++ b/dbms/src/Common/ColumnsHashing.h @@ -1,14 +1,17 @@ #pragma once -#include -#include -#include -#include -#include + + +#include #include -#include #include #include -#include + +#include +#include +#include + +#include +#include namespace DB { @@ -32,118 +35,12 @@ public: using HashMethodContextPtr = std::shared_ptr; -template -struct MappedTraits -{ - using Type = void *; - static Type getMapped(T &) { return nullptr; } - static T & getKey(T & key) { return key; } -}; - -template -struct MappedTraits> -{ - using Type = Second *; - static Type getMapped(PairNoInit & value) { return &value.second; } - static First & getKey(PairNoInit & value) { return value.first; } -}; - -template -struct HashTableTraits -{ - using Value = typename Data::value_type; - using Mapped = typename MappedTraits::Type; - - static Mapped getMapped(Value & value) { return MappedTraits::getMapped(value); } - static auto & getKey(Value & value) { return MappedTraits::getKey(value); } -}; - -template -struct LastElementCache -{ - static constexpr bool consecutive_keys_optimization = consecutive_keys_optimization_; - using Value = typename HashTableTraits::Value; - Value value; - bool empty = true; - bool found = false; - - auto getMapped() { return HashTableTraits::getMapped(value); } - auto & getKey() { return HashTableTraits::getKey(value); } -}; - -template -struct LastElementCache -{ - static constexpr bool consecutive_keys_optimization = false; -}; - -template -inline ALWAYS_INLINE typename HashTableTraits::Value & emplaceKeyImpl( - Key key, Data & data, bool & inserted, Cache & cache [[maybe_unused]]) -{ - if constexpr (Cache::consecutive_keys_optimization) - { - if (!cache.empty && cache.found && cache.getKey() == key) - { - inserted = false; - return cache.value; - } - } - - typename Data::iterator it; - data.emplace(key, it, inserted); - auto & value = *it; - - if constexpr (Cache::consecutive_keys_optimization) - { - cache.value = value; - cache.empty = false; - cache.found = true; - } - - return value; -} - -template -inline ALWAYS_INLINE typename HashTableTraits::Mapped findKeyImpl( - Key key, Data & data, bool & found, Cache & cache [[maybe_unused]]) -{ - if constexpr (Cache::consecutive_keys_optimization) - { - if (!cache.empty && cache.getKey() == key) - { - found = cache.found; - return found ? cache.getMapped() : nullptr; - } - } - - auto it = data.find(key); - - found = it != data.end(); - auto mapped = found ? HashTableTraits::getMapped(*it) - : nullptr; - - if constexpr (Cache::consecutive_keys_optimization) - { - if (found) - cache.value = *it; - else - cache.getKey() = key; - - cache.empty = false; - cache.found = found; - } - - return mapped; -} - - /// For the case where there is one numeric key. -template /// UInt8/16/32/64 for any type with corresponding bit width. -struct HashMethodOneNumber +template /// UInt8/16/32/64 for any type with corresponding bit width. +struct HashMethodOneNumber : public columns_hashing_impl::HashMethodBase { + using Base = columns_hashing_impl::HashMethodBase; const char * vec; - LastElementCache last_elem_cache; /// If the keys of a fixed length then key_sizes contains their lengths, empty otherwise. HashMethodOneNumber(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) @@ -158,27 +55,20 @@ struct HashMethodOneNumber /// Emplace key into HashTable or HashMap. If Data is HashMap, returns ptr to value, otherwise nullptr. template - ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey( + ALWAYS_INLINE typename Base::EmplaceResult emplaceKey( Data & data, /// HashTable size_t row, /// From which row of the block insert the key - bool & inserted, Arena & /*pool*/) /// For Serialized method, key may be placed in pool. { - return HashTableTraits::getMapped(emplaceKeyImpl(getKey(row), data, inserted, last_elem_cache)); + typename Data::iterator it; + return Base::emplaceKeyImpl(getKey(row), data, it); } /// Find key into HashTable or HashMap. If Data is HashMap and key was found, returns ptr to value, otherwise nullptr. template - ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & /*pool*/) + ALWAYS_INLINE typename Base::FindResult findKey(Data & data, size_t row, Arena & /*pool*/) { - return findKeyImpl(getKey(row), data, found, last_elem_cache); - } - - /// Insert the key from the hash table into columns. - template - static void insertKeyIntoColumns(const Value & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) - { - static_cast(key_columns[0].get())->insertRawData(reinterpret_cast(&value.first)); + return Base::findKeyImpl(getKey(row), data); } /// Get hash value of row. @@ -189,34 +79,24 @@ struct HashMethodOneNumber } /// Get StringRef from value which can be inserted into column. - template static StringRef getValueRef(const Value & value) { return StringRef(reinterpret_cast(&value.first), sizeof(value.first)); } - /// Cache last result if key was inserted. - template - ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped mapped) - { - *last_elem_cache.getMapped() = mapped; - } - protected: - template static ALWAYS_INLINE void onNewKey(Value & /*value*/, Arena & /*pool*/) {} }; /// For the case where there is one string key. -template -struct HashMethodString +template +struct HashMethodString : public columns_hashing_impl::HashMethodBase { + using Base = columns_hashing_impl::HashMethodBase; const IColumn::Offset * offsets; const UInt8 * chars; - LastElementCache last_elem_cache; - HashMethodString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) { const IColumn & column = *key_columns[0]; @@ -230,28 +110,23 @@ struct HashMethodString StringRef getKey(size_t row) const { return StringRef(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1); } template - ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row, bool & inserted, Arena & pool) + ALWAYS_INLINE typename Base::EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool) { - auto & value = emplaceKeyImpl(getKey(row), data, inserted, last_elem_cache); - if (inserted) + auto key = getKey(row); + typename Data::iterator it; + auto result = Base::emplaceKeyImpl(key, data, it); + if (result.isInserted()) { - auto & key = HashTableTraits::getKey(value); if (key.size) - key.data = pool.insert(key.data, key.size); + it->first.data = pool.insert(key.data, key.size); } - return HashTableTraits::getMapped(value); + return result; } template - ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & /*pool*/) + ALWAYS_INLINE typename Base::FindResult findKey(Data & data, size_t row, Arena & /*pool*/) { - return findKeyImpl(getKey(row), data, found, last_elem_cache); - } - - template - static void insertKeyIntoColumns(const Value & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) - { - key_columns[0]->insertData(value.first.data, value.first.size); + return Base::findKeyImpl(getKey(row), data); } template @@ -260,20 +135,12 @@ struct HashMethodString return data.hash(getKey(row)); } - template static StringRef getValueRef(const Value & value) { return StringRef(value.first.data, value.first.size); } - template - ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped mapped) - { - *last_elem_cache.getMapped() = mapped; - } - protected: - template static ALWAYS_INLINE void onNewKey(Value & value, Arena & pool) { if (value.first.size) @@ -283,14 +150,13 @@ protected: /// For the case where there is one fixed-length string key. -template -struct HashMethodFixedString +template +struct HashMethodFixedString : public columns_hashing_impl::HashMethodBase { + using Base = columns_hashing_impl::HashMethodBase; size_t n; const ColumnFixedString::Chars * chars; - LastElementCache last_elem_cache; - HashMethodFixedString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) { const IColumn & column = *key_columns[0]; @@ -304,27 +170,21 @@ struct HashMethodFixedString StringRef getKey(size_t row) const { return StringRef(&(*chars)[row * n], n); } template - ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row, bool & inserted, Arena & pool) + ALWAYS_INLINE typename Base::EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool) { - auto & value = emplaceKeyImpl(getKey(row), data, inserted, last_elem_cache); - if (inserted) - { - auto & key = HashTableTraits::getKey(value); - key.data = pool.insert(key.data, key.size); - } - return HashTableTraits::getMapped(value); + auto key = getKey(row); + typename Data::iterator it; + auto res = Base::emplaceKeyImpl(key, data, it); + if (res.isInserted()) + it->first.data = pool.insert(key.data, key.size); + + return res; } template - ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & /*pool*/) + ALWAYS_INLINE typename Base::FindResult findKey(Data & data, size_t row, Arena & /*pool*/) { - return findKeyImpl(getKey(row), data, found, last_elem_cache); - } - - template - static void insertKeyIntoColumns(const Value & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) - { - key_columns[0]->insertData(value.first.data, value.first.size); + return Base::findKeyImpl(getKey(row), data); } template @@ -333,20 +193,12 @@ struct HashMethodFixedString return data.hash(getKey(row)); } - template static StringRef getValueRef(const Value & value) { return StringRef(value.first.data, value.first.size); } - template - ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped mapped) - { - *last_elem_cache.getMapped() = mapped; - } - protected: - template static ALWAYS_INLINE void onNewKey(Value & value, Arena & pool) { value.first.data = pool.insert(value.first.data, value.first.size); @@ -400,12 +252,24 @@ private: Cache cache; }; + /// Single low cardinality column. -template +template struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod { using Base = SingleColumnMethod; + enum class VisitValue + { + Empty = 0, + Found = 1, + NotFound = 2, + }; + + static constexpr bool has_mapped = !std::is_same::value; + using EmplaceResult = columns_hashing_impl::EmplaceResultImpl; + using FindResult = columns_hashing_impl::FindResultImpl; + static HashMethodContextPtr createContext(const HashMethodContext::Settings & settings) { return std::make_shared(settings); @@ -421,7 +285,8 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod ColumnPtr dictionary_holder; /// Cache AggregateDataPtr for current column in order to decrease the number of hash table usages. - PaddedPODArray aggregate_data_cache; + columns_hashing_impl::MappedCache mapped_cache; + PaddedPODArray visit_cache; /// If initialized column is nullable. bool is_nullable = false; @@ -495,8 +360,11 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod } } - AggregateDataPtr default_data = nullptr; - aggregate_data_cache.assign(key_columns[0]->size(), default_data); + if constexpr (has_mapped) + mapped_cache.resize(key_columns[0]->size()); + + VisitValue empty(VisitValue::Empty); + visit_cache.assign(key_columns[0]->size(), empty); size_of_index_type = column->getSizeOfIndexType(); positions = column->getIndexesPtr().get(); @@ -521,41 +389,45 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod } template - ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row_, bool & inserted, Arena & pool) + ALWAYS_INLINE EmplaceResult emplaceKey(Data & data, size_t row_, Arena & pool) { size_t row = getIndexAt(row_); if (is_nullable && row == 0) { - inserted = !data.hasNullKeyData(); - data.hasNullKeyData() = true; - return &data.getNullKeyData(); + visit_cache[row] = VisitValue::Found; + if constexpr (has_mapped) + return EmplaceResult(data.getNullKeyData(), mapped_cache[0], !data.hasNullKeyData()); + else + return EmplaceResult(!data.hasNullKeyData()); } - if constexpr (use_cache) + if (visit_cache[row] == VisitValue::Found) { - if (aggregate_data_cache[row]) - { - inserted = false; - return &aggregate_data_cache[row]; - } + if constexpr (has_mapped) + return EmplaceResult(mapped_cache[row], mapped_cache[row], false); + else + return EmplaceResult(false); } - Sizes key_sizes; auto key = getKey(row_); + bool inserted = false; typename Data::iterator it; if (saved_hash) data.emplace(key, it, inserted, saved_hash[row]); else data.emplace(key, it, inserted); + visit_cache[row] = VisitValue::Found; + if (inserted) Base::onNewKey(*it, pool); - else if constexpr (use_cache) - aggregate_data_cache[row] = it->second; - return HashTableTraits::getMapped(*it); + if constexpr (has_mapped) + return EmplaceResult(it->second, mapped_cache[row], inserted); + else + return EmplaceResult(inserted); } ALWAYS_INLINE bool isNullAt(size_t i) @@ -566,25 +438,25 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod return getIndexAt(i) == 0; } - template - ALWAYS_INLINE void cacheData(size_t i, Mapped mapped) - { - size_t row = getIndexAt(i); - aggregate_data_cache[row] = mapped; - } - template - ALWAYS_INLINE typename HashTableTraits::Mapped findFromRow(Data & data, size_t row_, bool & found, Arena &) + ALWAYS_INLINE FindResult findFromRow(Data & data, size_t row_, Arena &) { size_t row = getIndexAt(row_); if (is_nullable && row == 0) - return data.hasNullKeyData() ? &data.getNullKeyData() : nullptr; - - if constexpr (use_cache) { - if (aggregate_data_cache[row]) - return &aggregate_data_cache[row]; + if constexpr (has_mapped) + return FindResult(data.hasNullKeyData() ? data.getNullKeyData() : Mapped(), data.hasNullKeyData()); + else + return FindResult(data.hasNullKeyData()); + } + + if (visit_cache[row] != VisitValue::Empty) + { + if constexpr (has_mapped) + return FindResult(mapped_cache[row], visit_cache[row] == VisitValue::Found); + else + return FindResult(visit_cache[row] == VisitValue::Found); } auto key = getKey(row_); @@ -595,14 +467,19 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod else it = data.find(key); - found = it != data.end(); - if constexpr (use_cache) + bool found = it != data.end(); + visit_cache[row] = found ? VisitValue::Found : VisitValue::NotFound; + + if constexpr (has_mapped) { if (found) - aggregate_data_cache[row] = it->second; + mapped_cache[row] = it->second; } - return typename HashTableTraits::getMapped(*it); + if constexpr (has_mapped) + return FindResult(mapped_cache[row], found); + else + return FindResult(found); } template @@ -614,108 +491,9 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod return Base::getHash(data, row, pool); } - - template - static void insertKeyIntoColumns(const Value & value, MutableColumns & key_columns_low_cardinality, const Sizes & /*key_sizes*/) - { - auto ref = Base::getValueRef(value); - static_cast(key_columns_low_cardinality[0].get())->insertData(ref.data, ref.size); - } }; -namespace columns_hashing_impl -{ - -/// This class is designed to provide the functionality that is required for -/// supporting nullable keys in HashMethodKeysFixed. If there are -/// no nullable keys, this class is merely implemented as an empty shell. -template -class BaseStateKeysFixed; - -/// Case where nullable keys are supported. -template -class BaseStateKeysFixed -{ -protected: - void init(const ColumnRawPtrs & key_columns) - { - null_maps.reserve(key_columns.size()); - actual_columns.reserve(key_columns.size()); - - for (const auto & col : key_columns) - { - if (col->isColumnNullable()) - { - const auto & nullable_col = static_cast(*col); - actual_columns.push_back(&nullable_col.getNestedColumn()); - null_maps.push_back(&nullable_col.getNullMapColumn()); - } - else - { - actual_columns.push_back(col); - null_maps.push_back(nullptr); - } - } - } - - /// Return the columns which actually contain the values of the keys. - /// For a given key column, if it is nullable, we return its nested - /// column. Otherwise we return the key column itself. - inline const ColumnRawPtrs & getActualColumns() const - { - return actual_columns; - } - - /// Create a bitmap that indicates whether, for a particular row, - /// a key column bears a null value or not. - KeysNullMap createBitmap(size_t row) const - { - KeysNullMap bitmap{}; - - for (size_t k = 0; k < null_maps.size(); ++k) - { - if (null_maps[k] != nullptr) - { - const auto & null_map = static_cast(*null_maps[k]).getData(); - if (null_map[row] == 1) - { - size_t bucket = k / 8; - size_t offset = k % 8; - bitmap[bucket] |= UInt8(1) << offset; - } - } - } - - return bitmap; - } - -private: - ColumnRawPtrs actual_columns; - ColumnRawPtrs null_maps; -}; - -/// Case where nullable keys are not supported. -template -class BaseStateKeysFixed -{ -protected: - void init(const ColumnRawPtrs & columns) { actual_columns = columns; } - - const ColumnRawPtrs & getActualColumns() const { return actual_columns; } - - KeysNullMap createBitmap(size_t) const - { - throw Exception{"Internal error: calling createBitmap() for non-nullable keys" - " is forbidden", ErrorCodes::LOGICAL_ERROR}; - } - -private: - ColumnRawPtrs actual_columns; -}; - -} - // Optional mask for low cardinality columns. template struct LowCardinalityKeys @@ -729,11 +507,11 @@ template <> struct LowCardinalityKeys {}; /// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits. -template -struct HashMethodKeysFixed : private columns_hashing_impl::BaseStateKeysFixed +template +struct HashMethodKeysFixed + : private columns_hashing_impl::BaseStateKeysFixed + , public columns_hashing_impl::HashMethodBase { - using Key = typename TData::key_type; - static constexpr bool has_nullable_keys = has_nullable_keys_; static constexpr bool has_low_cardinality = has_low_cardinality_; @@ -741,9 +519,8 @@ struct HashMethodKeysFixed : private columns_hashing_impl::BaseStateKeysFixed last_elem_cache; - using Base = columns_hashing_impl::BaseStateKeysFixed; + using BaseHashed = columns_hashing_impl::HashMethodBase; HashMethodKeysFixed(const ColumnRawPtrs & key_columns, const Sizes & key_sizes, const HashMethodContextPtr &) : key_sizes(std::move(key_sizes)), keys_size(key_columns.size()) @@ -789,21 +566,16 @@ struct HashMethodKeysFixed : private columns_hashing_impl::BaseStateKeysFixed - ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row, bool & inserted, Arena & /*pool*/) + ALWAYS_INLINE typename BaseHashed::EmplaceResult emplaceKey(Data & data, size_t row, Arena & /*pool*/) { - return HashTableTraits::getMapped(emplaceKeyImpl(getKey(row), data, inserted, last_elem_cache)); + typename Data::iterator it; + return BaseHashed::emplaceKeyImpl(getKey(row), data, it); } template - ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & /*pool*/) + ALWAYS_INLINE typename BaseHashed::FindResult findKey(Data & data, size_t row, Arena & /*pool*/) { - return findKeyImpl(getKey(row), data, found, last_elem_cache); - } - - template - static StringRef getValueRef(const Value & value) - { - return StringRef(value.first.data, value.first.size); + return BaseHashed::findKeyImpl(getKey(row), data); } template @@ -811,12 +583,6 @@ struct HashMethodKeysFixed : private columns_hashing_impl::BaseStateKeysFixed - ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped mapped) - { - *last_elem_cache.getMapped() = mapped; - } }; /** Hash by concatenating serialized key values. @@ -824,12 +590,12 @@ struct HashMethodKeysFixed : private columns_hashing_impl::BaseStateKeysFixed -struct HashMethodSerialized +template +struct HashMethodSerialized : public columns_hashing_impl::HashMethodBase { + using Base = columns_hashing_impl::HashMethodBase; ColumnRawPtrs key_columns; size_t keys_size; - LastElementCache last_elem_cache; HashMethodSerialized(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) : key_columns(key_columns), keys_size(key_columns.size()) {} @@ -837,24 +603,25 @@ struct HashMethodSerialized static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; } template - ALWAYS_INLINE typename HashTableTraits::Mapped emplaceKey(Data & data, size_t row, bool & inserted, Arena & pool) + ALWAYS_INLINE typename Base::EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool) { auto key = getKey(row, pool); - auto & value = emplaceKeyImpl(key, data, inserted, last_elem_cache); - if (!inserted) + typename Data::iterator it; + auto res = Base::emplaceKeyImpl(key, data, it); + if (!res.isInserted()) pool.rollback(key.size); - return HashTableTraits::getMapped(value); + return res; } template - ALWAYS_INLINE typename HashTableTraits::Mapped findKey(Data & data, size_t row, bool & found, Arena & pool) + ALWAYS_INLINE typename Base::FindResult findKey(Data & data, size_t row, Arena & pool) { auto key = getKey(row, pool); - auto mapped = findKeyImpl(key, data, found, last_elem_cache); + auto res = Base::findKeyImpl(key, data); pool.rollback(key.size); - return mapped; + return res; } template @@ -867,9 +634,6 @@ struct HashMethodSerialized return hash; } - template - ALWAYS_INLINE void cacheData(size_t /*row*/, Mapped /*mapped*/) {} - protected: ALWAYS_INLINE StringRef getKey(size_t row, Arena & pool) const { diff --git a/dbms/src/Common/ColumnsHashingImpl.h b/dbms/src/Common/ColumnsHashingImpl.h new file mode 100644 index 00000000000..565940b3338 --- /dev/null +++ b/dbms/src/Common/ColumnsHashingImpl.h @@ -0,0 +1,276 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace ColumnsHashing +{ + +namespace columns_hashing_impl +{ + +template +struct LastElementCache +{ + static constexpr bool consecutive_keys_optimization = consecutive_keys_optimization_; + Value value; + bool empty = true; + bool found = false; + + bool check(const Value & value_) { return !empty && value == value_; } + + template + bool check(const Key & key) { return !empty && value.first == key; } +}; + +template +struct LastElementCache +{ + static constexpr bool consecutive_keys_optimization = false; +}; + +template +class EmplaceResultImpl +{ + Mapped & value; + Mapped & cached_value; + bool inserted; + +public: + EmplaceResultImpl(Mapped & value, Mapped & cached_value, bool inserted) + : value(value), cached_value(cached_value), inserted(inserted) {} + + bool isInserted() const { return inserted; } + const auto & getMapped() const { return value; } + void setMapped(const Mapped & mapped) { value = cached_value = mapped; } +}; + +template <> +class EmplaceResultImpl +{ + bool inserted; + +public: + explicit EmplaceResultImpl(bool inserted) : inserted(inserted) {} + bool isInserted() const { return inserted; } +}; + +template +class FindResultImpl +{ + Mapped value; + bool found; + +public: + FindResultImpl(Mapped value, bool found) : value(value), found(found) {} + bool isFound() const { return found; } + const Mapped & getMapped() const { return value; } +}; + +template <> +class FindResultImpl +{ + bool found; + +public: + explicit FindResultImpl(bool found) : found(found) {} + bool isFound() const { return found; } +}; + +template +struct HashMethodBase +{ + using EmplaceResult = EmplaceResultImpl; + using FindResult = FindResultImpl; + static constexpr bool has_mapped = !std::is_same::value; + using Cache = LastElementCache; + +protected: + Cache cache; + + HashMethodBase() + { + if constexpr (has_mapped && consecutive_keys_optimization) + { + /// Init PairNoInit elements. + cache.value.second = Mapped(); + using Key = decltype(cache.value.first); + cache.value.first = Key(); + } + } + + template + ALWAYS_INLINE EmplaceResult emplaceKeyImpl(Key key, Data & data, typename Data::iterator & it) + { + if constexpr (Cache::consecutive_keys_optimization) + { + if (cache.found && cache.check(key)) + { + if constexpr (has_mapped) + return EmplaceResult(cache.value.second, cache.value.second, false); + else + return EmplaceResult(false); + } + } + + bool inserted = false; + data.emplace(key, it, inserted); + Mapped * cached = &it->second; + + if constexpr (consecutive_keys_optimization) + { + cache.value = *it; + cache.found = true; + cache.empty = false; + cached = &cache.value.second; + } + + if constexpr (has_mapped) + return EmplaceResult(it->second, *cached, inserted); + else + return EmplaceResult(inserted); + } + + template + ALWAYS_INLINE FindResult findKeyImpl(Key key, Data & data) + { + if constexpr (Cache::consecutive_keys_optimization) + { + if (cache.check(key)) + { + if constexpr (has_mapped) + return FindResult(cache.found ? cache.value.second : Mapped(), cache.found); + else + return FindResult(cache.found); + } + } + + auto it = data.find(key); + bool found = it != data.end(); + + if constexpr (consecutive_keys_optimization) + { + cache.found = found; + cache.empty = false; + + if (found) + cache.value = *it; + else + { + if constexpr (has_mapped) + cache.value.first = key; + else + cache.value = key; + } + } + + if constexpr (has_mapped) + return FindResult(found ? it->second : Mapped(), found); + else + return FindResult(found); + } +}; + + +template +struct MappedCache : public PaddedPODArray {}; + +template <> +struct MappedCache {}; + + +/// This class is designed to provide the functionality that is required for +/// supporting nullable keys in HashMethodKeysFixed. If there are +/// no nullable keys, this class is merely implemented as an empty shell. +template +class BaseStateKeysFixed; + +/// Case where nullable keys are supported. +template +class BaseStateKeysFixed +{ +protected: + void init(const ColumnRawPtrs & key_columns) + { + null_maps.reserve(key_columns.size()); + actual_columns.reserve(key_columns.size()); + + for (const auto & col : key_columns) + { + if (col->isColumnNullable()) + { + const auto & nullable_col = static_cast(*col); + actual_columns.push_back(&nullable_col.getNestedColumn()); + null_maps.push_back(&nullable_col.getNullMapColumn()); + } + else + { + actual_columns.push_back(col); + null_maps.push_back(nullptr); + } + } + } + + /// Return the columns which actually contain the values of the keys. + /// For a given key column, if it is nullable, we return its nested + /// column. Otherwise we return the key column itself. + inline const ColumnRawPtrs & getActualColumns() const + { + return actual_columns; + } + + /// Create a bitmap that indicates whether, for a particular row, + /// a key column bears a null value or not. + KeysNullMap createBitmap(size_t row) const + { + KeysNullMap bitmap{}; + + for (size_t k = 0; k < null_maps.size(); ++k) + { + if (null_maps[k] != nullptr) + { + const auto & null_map = static_cast(*null_maps[k]).getData(); + if (null_map[row] == 1) + { + size_t bucket = k / 8; + size_t offset = k % 8; + bitmap[bucket] |= UInt8(1) << offset; + } + } + } + + return bitmap; + } + +private: + ColumnRawPtrs actual_columns; + ColumnRawPtrs null_maps; +}; + +/// Case where nullable keys are not supported. +template +class BaseStateKeysFixed +{ +protected: + void init(const ColumnRawPtrs & columns) { actual_columns = columns; } + + const ColumnRawPtrs & getActualColumns() const { return actual_columns; } + + KeysNullMap createBitmap(size_t) const + { + throw Exception{"Internal error: calling createBitmap() for non-nullable keys" + " is forbidden", ErrorCodes::LOGICAL_ERROR}; + } + +private: + ColumnRawPtrs actual_columns; +}; + +} + +} + +} diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 785345f9400..e26d94f53e4 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -609,20 +609,34 @@ void NO_INLINE Aggregator::executeImplCase( /// NOTE When editing this code, also pay attention to SpecializedAggregator.h. /// For all rows. - AggregateDataPtr value = nullptr; for (size_t i = 0; i < rows; ++i) { - bool inserted = false; /// Inserted a new key, or was this key already? - - AggregateDataPtr * aggregate_data = nullptr; + AggregateDataPtr aggregate_data = nullptr; if constexpr (!no_more_keys) /// Insert. - aggregate_data = state.emplaceKey(method.data, i, inserted, *aggregates_pool); + { + auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool); + + /// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. + if (emplace_result.isInserted()) + { + /// exception-safety - if you can not allocate memory or create states, then destructors will not be called. + emplace_result.setMapped(nullptr); + + aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data); + + emplace_result.setMapped(aggregate_data); + } + else + aggregate_data = emplace_result.getMapped(); + } else { /// Add only if the key already exists. - bool found = false; - aggregate_data = state.findKey(method.data, i, found, *aggregates_pool); + auto find_result = state.findKey(method.data, i, *aggregates_pool); + if (find_result.isFound()) + aggregate_data = find_result.getMapped(); } /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys. @@ -631,20 +645,7 @@ void NO_INLINE Aggregator::executeImplCase( if (!aggregate_data && !overflow_row) continue; - /// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. - if (inserted) - { - /// exception-safety - if you can not allocate memory or create states, then destructors will not be called. - *aggregate_data = nullptr; - - AggregateDataPtr place = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); - createAggregateStates(place); - *aggregate_data = place; - - state.cacheData(i, place); - } - - value = aggregate_data ? *aggregate_data : overflow_row; + AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row; /// Add values to the aggregate functions. for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) @@ -1951,17 +1952,28 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( size_t rows = block.rows(); for (size_t i = 0; i < rows; ++i) { - typename Table::iterator it; - AggregateDataPtr * aggregate_data = nullptr; - - bool inserted = false; /// Inserted a new key, or was this key already? + AggregateDataPtr aggregate_data = nullptr; if (!no_more_keys) - aggregate_data = state.emplaceKey(data, i, inserted, *aggregates_pool); + { + auto emplace_result = state.emplaceKey(data, i, *aggregates_pool); + if (emplace_result.isInserted()) + { + emplace_result.setMapped(nullptr); + + aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data); + + emplace_result.setMapped(aggregate_data); + } + else + aggregate_data = emplace_result.getMapped(); + } else { - bool found; - aggregate_data = state.findKey(data, i, found, *aggregates_pool); + auto find_result = state.findKey(data, i, *aggregates_pool); + if (find_result.isFound()) + aggregate_data = find_result.getMapped(); } /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys. @@ -1970,19 +1982,7 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( if (!aggregate_data && !overflow_row) continue; - /// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. - if (inserted) - { - *aggregate_data = nullptr; - - AggregateDataPtr place = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); - createAggregateStates(place); - *aggregate_data = place; - - state.cacheData(i, place); - } - - AggregateDataPtr value = aggregate_data ? *aggregate_data : overflow_row; + AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row; /// Merge state of aggregate functions. for (size_t j = 0; j < params.aggregates_size; ++j) diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 0b40f4e6a25..9112e0265cb 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -158,7 +158,7 @@ struct AggregationMethodOneNumber AggregationMethodOneNumber(const Other & other) : data(other.data) {} /// To use one `Method` in different threads, use different `State`. - using State = ColumnsHashing::HashMethodOneNumber; + using State = ColumnsHashing::HashMethodOneNumber; /// Use optimization for low cardinality. static const bool low_cardinality_optimization = false; @@ -188,7 +188,7 @@ struct AggregationMethodString template AggregationMethodString(const Other & other) : data(other.data) {} - using State = ColumnsHashing::HashMethodString; + using State = ColumnsHashing::HashMethodString; static const bool low_cardinality_optimization = false; @@ -216,7 +216,7 @@ struct AggregationMethodFixedString template AggregationMethodFixedString(const Other & other) : data(other.data) {} - using State = ColumnsHashing::HashMethodFixedString; + using State = ColumnsHashing::HashMethodFixedString; static const bool low_cardinality_optimization = false; @@ -246,7 +246,7 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod template explicit AggregationMethodSingleLowCardinalityColumn(const Other & other) : Base(other) {} - using State = ColumnsHashing::HashMethodSingleLowCardinalityColumn; + using State = ColumnsHashing::HashMethodSingleLowCardinalityColumn; static const bool low_cardinality_optimization = true; @@ -277,7 +277,7 @@ struct AggregationMethodKeysFixed template AggregationMethodKeysFixed(const Other & other) : data(other.data) {} - using State = ColumnsHashing::HashMethodKeysFixed; + using State = ColumnsHashing::HashMethodKeysFixed; static const bool low_cardinality_optimization = false; @@ -355,7 +355,7 @@ struct AggregationMethodSerialized template AggregationMethodSerialized(const Other & other) : data(other.data) {} - using State = ColumnsHashing::HashMethodSerialized; + using State = ColumnsHashing::HashMethodSerialized; static const bool low_cardinality_optimization = false; From 95f2f3faebbee6e1c5fbed940efb246114330419 Mon Sep 17 00:00:00 2001 From: Odin Hultgren Van Der Horst Date: Wed, 23 Jan 2019 11:48:30 +0100 Subject: [PATCH 018/324] Removed old comment --- contrib/librdkafka-cmake/config.h | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/librdkafka-cmake/config.h b/contrib/librdkafka-cmake/config.h index 0de2ca1a383..979d66cdca5 100644 --- a/contrib/librdkafka-cmake/config.h +++ b/contrib/librdkafka-cmake/config.h @@ -10,7 +10,6 @@ #define ENABLE_SHAREDPTR_DEBUG 0 #define ENABLE_LZ4_EXT 1 #define ENABLE_SSL 1 -//#define WITH_HDRHISTOGRAM 0 //#define ENABLE_SASL 1 #define MKL_APP_NAME "librdkafka" #define MKL_APP_DESC_ONELINE "The Apache Kafka C/C++ library" From 5154814bf941eb5b9b3b6262136d6f6fc27294fd Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 18 Jan 2019 19:30:35 +0300 Subject: [PATCH 019/324] try to get rid of IAST::range #4058 --- dbms/src/Interpreters/ActionsVisitor.cpp | 15 +++++----- dbms/src/Interpreters/ActionsVisitor.h | 10 ++----- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 4 +-- .../Interpreters/InterpreterCreateQuery.cpp | 14 ++++----- .../LogicalExpressionsOptimizer.cpp | 8 ----- dbms/src/Interpreters/PreparedSets.h | 20 +++++++++++++ dbms/src/Interpreters/QueryNormalizer.h | 1 + dbms/src/Interpreters/executeQuery.cpp | 14 +++++---- dbms/src/Interpreters/getClusterName.cpp | 14 +++++++-- dbms/src/Parsers/ASTIdentifier.h | 1 - dbms/src/Parsers/ExpressionElementParsers.cpp | 6 ---- dbms/src/Parsers/ExpressionListParsers.cpp | 30 ------------------- dbms/src/Parsers/IAST.h | 8 +---- dbms/src/Parsers/IParserBase.cpp | 2 -- dbms/src/Storages/MergeTree/KeyCondition.cpp | 21 +++++++------ .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../ReplicatedMergeTreeMutationEntry.h | 1 + dbms/src/Storages/SelectQueryInfo.h | 12 +------- 18 files changed, 75 insertions(+), 108 deletions(-) create mode 100644 dbms/src/Interpreters/PreparedSets.h diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 8d679a4cb58..b53d166ddb4 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -131,7 +131,7 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool SetPtr set = std::make_shared(size_limits, create_ordered_set); set->createFromAST(set_element_types, elements_ast, context); - prepared_sets[right_arg->range] = std::move(set); + prepared_sets[right_arg->getTreeHash()] = std::move(set); } static String getUniqueName(const Block & block, const String & prefix) @@ -380,12 +380,12 @@ void ActionsVisitor::visit(const ASTPtr & ast) /// Select the name in the next cycle. argument_names.emplace_back(); } - else if (prepared_sets.count(child->range) && functionIsInOrGlobalInOperator(node->name) && arg == 1) + else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_sets.count(child->getTreeHash())) { ColumnWithTypeAndName column; column.type = std::make_shared(); - const SetPtr & set = prepared_sets[child->range]; + const SetPtr & set = prepared_sets[child->getTreeHash()]; /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). @@ -530,9 +530,10 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc */ const IAST & args = *node->arguments; const ASTPtr & arg = args.children.at(1); + IAST::Hash tree_hash = arg->getTreeHash(); /// Already converted. - if (prepared_sets.count(arg->range)) + if (prepared_sets.count(tree_hash)) return; /// If the subquery or table name for SELECT. @@ -552,7 +553,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc if (storage_set) { - prepared_sets[arg->range] = storage_set->getSet(); + prepared_sets[tree_hash] = storage_set->getSet(); return; } } @@ -566,7 +567,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc /// If you already created a Set with the same subquery / table. if (subquery_for_set.set) { - prepared_sets[arg->range] = subquery_for_set.set; + prepared_sets[tree_hash] = subquery_for_set.set; return; } @@ -612,7 +613,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc } subquery_for_set.set = set; - prepared_sets[arg->range] = set; + prepared_sets[tree_hash] = set; } else { diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 805b0ec02c4..d4a6d388fbc 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -1,6 +1,7 @@ #pragma once -#include +#include +#include #include @@ -10,13 +11,6 @@ namespace DB class Context; class ASTFunction; - -class Set; -using SetPtr = std::shared_ptr; -/// Will compare sets by their position in query string. It's possible because IAST::clone() doesn't chane IAST::range. -/// It should be taken into account when we want to change AST part which contains sets. -using PreparedSets = std::unordered_map; - class Join; using JoinPtr = std::shared_ptr; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 8c1e13815ec..6d39a1303a4 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -279,7 +279,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_ return; } - prepared_sets[subquery_or_table_name->range] = std::move(set); + prepared_sets[subquery_or_table_name->getTreeHash()] = std::move(set); } @@ -308,7 +308,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & { const ASTPtr & arg = args.children.at(1); - if (!prepared_sets.count(arg->range)) /// Not already prepared. + if (!prepared_sets.count(arg->getTreeHash())) /// Not already prepared. { if (typeid_cast(arg.get()) || isIdentifier(arg)) { diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 429c257b8d5..53bf85d2099 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -339,11 +339,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns) const auto column_declaration = std::make_shared(); column_declaration->name = column.name; - StringPtr type_name = std::make_shared(column.type->getName()); - auto pos = type_name->data(); - const auto end = pos + type_name->size(); - ParserIdentifierWithOptionalParameters storage_p; + String type_name = column.type->getName(); + auto pos = type_name.data(); + const auto end = pos + type_name.size(); column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0); columns_list->children.emplace_back(column_declaration); } @@ -362,11 +361,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->name = column.name; - StringPtr type_name = std::make_shared(column.type->getName()); - auto type_name_pos = type_name->data(); - const auto type_name_end = type_name_pos + type_name->size(); - ParserIdentifierWithOptionalParameters storage_p; + String type_name = column.type->getName(); + auto type_name_pos = type_name.data(); + const auto type_name_end = type_name_pos + type_name.size(); column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0); const auto defaults_it = columns.defaults.find(column.name); diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index 5226c96dce6..d7f04ff2a25 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -228,17 +228,10 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain /// Construct a list of literals `x1, ..., xN` from the string `expr = x1 OR ... OR expr = xN` ASTPtr value_list = std::make_shared(); - const char * min_range_first = nullptr; - const char * max_range_second = nullptr; for (const auto function : equality_functions) { const auto & operands = getFunctionOperands(function); value_list->children.push_back(operands[1]); - /// Get range min/max from all literals x1...xN, which will be used as tuple_functions' range - if (min_range_first == nullptr || min_range_first > operands[1]->range.first) - min_range_first = operands[1]->range.first; - if (max_range_second == nullptr || max_range_second < operands[1]->range.second) - max_range_second = operands[1]->range.second; } /// Sort the literals so that they are specified in the same order in the IN expression. @@ -260,7 +253,6 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain auto tuple_function = std::make_shared(); tuple_function->name = "tuple"; - tuple_function->range = StringRange(min_range_first, max_range_second); tuple_function->arguments = value_list; tuple_function->children.push_back(tuple_function->arguments); diff --git a/dbms/src/Interpreters/PreparedSets.h b/dbms/src/Interpreters/PreparedSets.h new file mode 100644 index 00000000000..4276e62a239 --- /dev/null +++ b/dbms/src/Interpreters/PreparedSets.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct ASTHalfHash +{ + UInt64 operator()(const IAST::Hash & ast_hash) const { return ast_hash.first; } +}; + +class Set; +using SetPtr = std::shared_ptr; + +using PreparedSets = std::unordered_map; + +} diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 3e55e0253e6..57f4645569a 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 3911e437fa6..728225fdb80 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -150,7 +150,7 @@ static std::tuple executeQueryImpl( ParserQuery parser(end, settings.enable_debug_queries); ASTPtr ast; - size_t query_size; + const char * query_end; /// Don't limit the size of internal queries. size_t max_query_size = 0; @@ -162,10 +162,11 @@ static std::tuple executeQueryImpl( /// TODO Parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size); - /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. - if (!(begin <= ast->range.first && ast->range.second <= end)) - throw Exception("Unexpected behavior: AST chars range is not inside source range", ErrorCodes::LOGICAL_ERROR); - query_size = ast->range.second - begin; + const auto * insert_query = dynamic_cast(ast.get()); + if (insert_query && insert_query->data) + query_end = insert_query->data; + else + query_end = end; } catch (...) { @@ -180,7 +181,8 @@ static std::tuple executeQueryImpl( throw; } - String query(begin, query_size); + /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. + String query(begin, query_end); BlockIO res; try diff --git a/dbms/src/Interpreters/getClusterName.cpp b/dbms/src/Interpreters/getClusterName.cpp index 4312da3e1a6..bc32e3dbea7 100644 --- a/dbms/src/Interpreters/getClusterName.cpp +++ b/dbms/src/Interpreters/getClusterName.cpp @@ -24,12 +24,22 @@ std::string getClusterName(const IAST & node) if (const ASTLiteral * ast_lit = typeid_cast(&node)) return ast_lit->value.safeGet(); + /// A hack to support hyphens in cluster names. if (const ASTFunction * ast_func = typeid_cast(&node)) { - if (!ast_func->range.first || !ast_func->range.second) + if (ast_func->name != "minus" || !ast_func->arguments || ast_func->arguments->children.size() < 2) throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS); - return String(ast_func->range.first, ast_func->range.second); + String name; + for (const auto & arg : ast_func->arguments->children) + { + if (name.empty()) + name += getClusterName(*arg); + else + name += "-" + getClusterName(*arg); + } + + return name; } throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS); diff --git a/dbms/src/Parsers/ASTIdentifier.h b/dbms/src/Parsers/ASTIdentifier.h index 4805a1f4fd5..a2b7f3ea555 100644 --- a/dbms/src/Parsers/ASTIdentifier.h +++ b/dbms/src/Parsers/ASTIdentifier.h @@ -19,7 +19,6 @@ public: : name(name_) , special(false) { - range = StringRange(name.data(), name.data() + name.size()); } /** Get the text that identifies this element. */ diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 9ba848dbe1a..a1fab382c36 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -672,8 +672,6 @@ bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - auto begin = pos; - if (!ParserKeyword("EXTRACT").ignore(pos, expected)) return false; @@ -734,14 +732,10 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp auto function = std::make_shared(); auto exp_list = std::make_shared(); - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = function_name; //"toYear"; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(expr); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; return true; diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index 9f17b3935f5..c732ce4f38e 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -138,7 +138,6 @@ static bool parseOperator(IParser::Pos & pos, const char * op, Expected & expect bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool first = true; - Pos begin = pos; while (1) { @@ -174,16 +173,12 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node return false; /// the first argument of the function is the previous element, the second is the next one - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = it[1]; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(node); exp_list->children.push_back(elem); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; /** special exception for the access operator to the element of the array `x[y]`, which * contains the infix part '[' and the suffix ''] '(specified as' [') @@ -243,8 +238,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp ASTPtr left; ASTPtr right; - Pos begin = pos; - if (!elem_parser.parse(pos, subject, expected)) return false; @@ -279,14 +272,10 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp args_le->children.emplace_back(subject); args_le->children.emplace_back(right); - f_ge->range.first = begin->begin; - f_ge->range.second = pos->begin; f_ge->name = "greaterOrEquals"; f_ge->arguments = args_ge; f_ge->children.emplace_back(f_ge->arguments); - f_le->range.first = begin->begin; - f_le->range.second = pos->begin; f_le->name = "lessOrEquals"; f_le->arguments = args_le; f_le->children.emplace_back(f_le->arguments); @@ -294,8 +283,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp args_and->children.emplace_back(f_ge); args_and->children.emplace_back(f_le); - f_and->range.first = begin->begin; - f_and->range.second = pos->begin; f_and->name = "and"; f_and->arguments = args_and; f_and->children.emplace_back(f_and->arguments); @@ -315,8 +302,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect ASTPtr elem_then; ASTPtr elem_else; - Pos begin = pos; - if (!elem_parser.parse(pos, elem_cond, expected)) return false; @@ -339,8 +324,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect /// function arguments auto exp_list = std::make_shared(); - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = "if"; function->arguments = exp_list; function->children.push_back(exp_list); @@ -348,8 +331,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect exp_list->children.push_back(elem_cond); exp_list->children.push_back(elem_then); exp_list->children.push_back(elem_else); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; } @@ -423,7 +404,6 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expe bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { /// try to find any of the valid operators - Pos begin = pos; const char ** it; for (it = operators; *it; it += 2) { @@ -471,15 +451,11 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex /// function arguments auto exp_list = std::make_shared(); - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = it[1]; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(elem); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; } @@ -595,8 +571,6 @@ bool ParserNullityChecking::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - Pos begin = pos; - /// If no INTERVAL keyword, go to nested parser. if (!ParserKeyword("INTERVAL").ignore(pos, expected)) return next_parser.parse(pos, node, expected); @@ -620,15 +594,11 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec auto exp_list = std::make_shared(); /// the first argument of the function is the previous element, the second is the next one - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = function_name; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(expr); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; return true; diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index a597d40080f..3c4b9dec007 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -7,7 +7,6 @@ #include #include -#include #include @@ -54,7 +53,6 @@ class IAST : public std::enable_shared_from_this { public: ASTs children; - StringRange range; SemanticPtr semantic; @@ -209,11 +207,7 @@ public: virtual void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const { - throw Exception("Unknown element in AST: " + getID() - + ((range.first && (range.second > range.first)) - ? " '" + std::string(range.first, range.second - range.first) + "'" - : ""), - ErrorCodes::UNKNOWN_ELEMENT_IN_AST); + throw Exception("Unknown element in AST: " + getID(), ErrorCodes::UNKNOWN_ELEMENT_IN_AST); } void cloneChildren(); diff --git a/dbms/src/Parsers/IParserBase.cpp b/dbms/src/Parsers/IParserBase.cpp index dffb1d7597d..ddbac8e92ee 100644 --- a/dbms/src/Parsers/IParserBase.cpp +++ b/dbms/src/Parsers/IParserBase.cpp @@ -22,8 +22,6 @@ bool IParserBase::parse(Pos & pos, ASTPtr & node, Expected & expected) node = nullptr; pos = begin; } - else if (node) - node->range = StringRange(begin, pos); return res; } diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index d386b865231..87d9f8def3d 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -635,18 +635,18 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo DataTypePtr key_expr_type; /// Type of expression containing key column size_t key_arg_pos; /// Position of argument with key column (non-const argument) - size_t key_column_num; /// Number of a key column (inside key_column_names array) + size_t key_column_num = -1; /// Number of a key column (inside key_column_names array) MonotonicFunctionsChain chain; bool is_set_const = false; bool is_constant_transformed = false; - if (prepared_sets.count(args[1]->range) - && tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->range], key_column_num)) - { - key_arg_pos = 0; - is_set_const = true; - } - else if (getConstant(args[1], block_with_constants, const_value, const_type) + if (prepared_sets.count(args[1]->getTreeHash()) + && tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->getTreeHash()], key_column_num)) + { + key_arg_pos = 0; + is_set_const = true; + } + else if (getConstant(args[1], block_with_constants, const_value, const_type) && isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain)) { key_arg_pos = 0; @@ -671,6 +671,9 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo else return false; + if (key_column_num == static_cast(-1)) + throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR); + std::string func_name = func->name; /// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5" @@ -1015,7 +1018,7 @@ bool KeyCondition::mayBeTrueInParallelogram(const std::vector & parallelo { auto in_func = typeid_cast(element.in_function.get()); const ASTs & args = typeid_cast(*in_func->arguments).children; - PreparedSets::const_iterator it = prepared_sets.find(args[1]->range); + PreparedSets::const_iterator it = prepared_sets.find(args[1]->getTreeHash()); if (in_func && it != prepared_sets.end()) { rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types)); diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 850c696b266..3eb9576b1c0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -334,7 +334,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) || (primary_key_columns.count(second_arg_name) && isConstant(args[0])) || (primary_key_columns.count(first_arg_name) - && (prepared_sets.count(args[1]->range) || typeid_cast(args[1].get())))) + && (typeid_cast(args[1].get()) || prepared_sets.count(args[1]->getTreeHash())))) return true; } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h index 68aab6fa021..8f253df3f10 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index 1aed0a98f8f..e68078a6712 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -1,24 +1,14 @@ #pragma once +#include #include -#include -#include namespace DB { -class IAST; -using ASTPtr = std::shared_ptr; - class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; -class Set; -using SetPtr = std::shared_ptr; - -/// Information about calculated sets in right hand side of IN. -using PreparedSets = std::unordered_map; - struct PrewhereInfo { /// Actions which are executed in order to alias columns are used for prewhere actions. From f3b4292fc691634473e5d4f6161c869db1813a8f Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 22 Jan 2019 15:33:56 +0300 Subject: [PATCH 020/324] add data types to the prepared_sets key for literals #4058 --- dbms/src/Core/Block.cpp | 12 ++ dbms/src/Core/Block.h | 1 + dbms/src/Interpreters/ActionsVisitor.cpp | 69 +++++---- dbms/src/Interpreters/ActionsVisitor.h | 9 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 59 ++++---- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- dbms/src/Interpreters/PreparedSets.h | 50 ++++++- dbms/src/Storages/MergeTree/KeyCondition.cpp | 139 ++++++++---------- dbms/src/Storages/MergeTree/KeyCondition.h | 13 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 5 +- .../MergeTree/MergeTreeWhereOptimizer.h | 1 - 11 files changed, 201 insertions(+), 159 deletions(-) diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index de0e831f37c..27b2cb81b09 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -427,6 +427,18 @@ Names Block::getNames() const } +DataTypes Block::getDataTypes() const +{ + DataTypes res; + res.reserve(columns()); + + for (const auto & elem : data) + res.push_back(elem.type); + + return res; +} + + template static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description) { diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index d8efc939ecd..d3ce0c67b79 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -82,6 +82,7 @@ public: const ColumnsWithTypeAndName & getColumnsWithTypeAndName() const; NamesAndTypesList getNamesAndTypesList() const; Names getNames() const; + DataTypes getDataTypes() const; /// Returns number of rows from first column in block, not equal to nullptr. If no columns, returns 0. size_t rows() const; diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index b53d166ddb4..1e8a17adecd 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -54,8 +54,9 @@ NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & [&](const NamesAndTypesList::value_type & val) { return val.name == name; }); } -void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set, - const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets) +SetPtr makeExplicitSet( + const ASTFunction * node, const Block & sample_block, bool create_ordered_set, + const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets) { const IAST & args = *node->arguments; @@ -65,6 +66,21 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool const ASTPtr & left_arg = args.children.at(0); const ASTPtr & right_arg = args.children.at(1); + const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type; + + DataTypes set_element_types = {left_arg_type}; + auto left_tuple_type = typeid_cast(left_arg_type.get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + + auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types); + if (prepared_sets.count(set_key)) + return prepared_sets.at(set_key); /// Already prepared. + auto getTupleTypeFromAst = [&context](const ASTPtr & tuple_ast) -> DataTypePtr { auto ast_function = typeid_cast(tuple_ast.get()); @@ -79,7 +95,6 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool return evaluateConstantExpression(tuple_ast, context).second; }; - const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type; const DataTypePtr & right_arg_type = getTupleTypeFromAst(right_arg); std::function getTupleDepth; @@ -94,15 +109,6 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool size_t left_tuple_depth = getTupleDepth(left_arg_type); size_t right_tuple_depth = getTupleDepth(right_arg_type); - DataTypes set_element_types = {left_arg_type}; - auto left_tuple_type = typeid_cast(left_arg_type.get()); - if (left_tuple_type && left_tuple_type->getElements().size() != 1) - set_element_types = left_tuple_type->getElements(); - - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); - ASTPtr elements_ast = nullptr; /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. @@ -131,7 +137,8 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool SetPtr set = std::make_shared(size_limits, create_ordered_set); set->createFromAST(set_element_types, elements_ast, context); - prepared_sets[right_arg->getTreeHash()] = std::move(set); + prepared_sets[set_key] = set; + return set; } static String getUniqueName(const Block & block, const String & prefix) @@ -305,6 +312,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) return; } + SetPtr prepared_set; if (functionIsInOrGlobalInOperator(node->name)) { /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). @@ -313,7 +321,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) if (!no_subqueries) { /// Transform tuple or subquery into a set. - makeSet(node, actions_stack.getSampleBlock()); + prepared_set = makeSet(node, actions_stack.getSampleBlock()); } else { @@ -380,23 +388,21 @@ void ActionsVisitor::visit(const ASTPtr & ast) /// Select the name in the next cycle. argument_names.emplace_back(); } - else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_sets.count(child->getTreeHash())) + else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_set) { ColumnWithTypeAndName column; column.type = std::make_shared(); - const SetPtr & set = prepared_sets[child->getTreeHash()]; - /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). - if (!set->empty()) + if (!prepared_set->empty()) column.name = getUniqueName(actions_stack.getSampleBlock(), "__set"); else column.name = child_column_name; if (!actions_stack.getSampleBlock().has(column.name)) { - column.column = ColumnSet::create(1, set); + column.column = ColumnSet::create(1, prepared_set); actions_stack.addAction(ExpressionAction::addColumn(column)); } @@ -522,7 +528,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) } } -void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_block) +SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_block) { /** You need to convert the right argument to a set. * This can be a table name, a value, a value enumeration, or a subquery. @@ -530,16 +536,15 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc */ const IAST & args = *node->arguments; const ASTPtr & arg = args.children.at(1); - IAST::Hash tree_hash = arg->getTreeHash(); - - /// Already converted. - if (prepared_sets.count(tree_hash)) - return; /// If the subquery or table name for SELECT. const ASTIdentifier * identifier = typeid_cast(arg.get()); if (typeid_cast(arg.get()) || identifier) { + auto set_key = PreparedSetKey::forSubquery(*arg); + if (prepared_sets.count(set_key)) + return prepared_sets.at(set_key); + /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). if (identifier) @@ -550,11 +555,10 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc if (table) { StorageSet * storage_set = dynamic_cast(table.get()); - if (storage_set) { - prepared_sets[tree_hash] = storage_set->getSet(); - return; + prepared_sets[set_key] = storage_set->getSet(); + return storage_set->getSet(); } } } @@ -567,8 +571,8 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc /// If you already created a Set with the same subquery / table. if (subquery_for_set.set) { - prepared_sets[tree_hash] = subquery_for_set.set; - return; + prepared_sets[set_key] = subquery_for_set.set; + return subquery_for_set.set; } SetPtr set = std::make_shared(set_size_limit, false); @@ -613,12 +617,13 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc } subquery_for_set.set = set; - prepared_sets[tree_hash] = set; + prepared_sets[set_key] = set; + return set; } else { /// An explicit enumeration of values in parentheses. - makeExplicitSet(node, sample_block, false, context, set_size_limit, prepared_sets); + return makeExplicitSet(node, sample_block, false, context, set_size_limit, prepared_sets); } } diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index d4a6d388fbc..12f9e1116c0 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -37,9 +37,10 @@ struct SubqueryForSet using SubqueriesForSets = std::unordered_map; -/// The case of an explicit enumeration of values. -void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set, - const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets); + /// The case of an explicit enumeration of values. +SetPtr makeExplicitSet( + const ASTFunction * node, const Block & sample_block, bool create_ordered_set, + const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets); /** For ActionsVisitor @@ -105,7 +106,7 @@ private: std::ostream * ostr; ScopeStack actions_stack; - void makeSet(const ASTFunction * node, const Block & sample_block); + SetPtr makeSet(const ASTFunction * node, const Block & sample_block); }; } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 6d39a1303a4..fd56c55e05f 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -258,20 +258,25 @@ void ExpressionAnalyzer::makeSetsForIndex() if (storage && select_query && storage->supportsIndexForIn()) { if (select_query->where_expression) - makeSetsForIndexImpl(select_query->where_expression, storage->getSampleBlock()); + makeSetsForIndexImpl(select_query->where_expression); if (select_query->prewhere_expression) - makeSetsForIndexImpl(select_query->prewhere_expression, storage->getSampleBlock()); + makeSetsForIndexImpl(select_query->prewhere_expression); } } void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name) { - BlockIO res = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {})->execute(); + auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); + if (prepared_sets.count(set_key)) + return; /// Already prepared. + + auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {}); + BlockIO res = interpreter_subquery->execute(); SetPtr set = std::make_shared(settings.size_limits_for_set, true); - set->setHeader(res.in->getHeader()); + while (Block block = res.in->read()) { /// If the limits have been exceeded, give up and let the default subquery processing actions take place. @@ -279,24 +284,24 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_ return; } - prepared_sets[subquery_or_table_name->getTreeHash()] = std::move(set); + prepared_sets[set_key] = std::move(set); } -void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block) +void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node) { for (auto & child : node->children) { - /// Don't descent into subqueries. + /// Don't descend into subqueries. if (typeid_cast(child.get())) continue; - /// Don't dive into lambda functions + /// Don't descend into lambda functions const ASTFunction * func = typeid_cast(child.get()); if (func && func->name == "lambda") continue; - makeSetsForIndexImpl(child, sample_block); + makeSetsForIndexImpl(child); } const ASTFunction * func = typeid_cast(node.get()); @@ -307,28 +312,24 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & if (storage && storage->mayBenefitFromIndexForIn(args.children.at(0))) { const ASTPtr & arg = args.children.at(1); - - if (!prepared_sets.count(arg->getTreeHash())) /// Not already prepared. + if (typeid_cast(arg.get()) || isIdentifier(arg)) { - if (typeid_cast(arg.get()) || isIdentifier(arg)) - { - if (settings.use_index_for_in_with_subqueries) - tryMakeSetForIndexFromSubquery(arg); - } - else - { - NamesAndTypesList temp_columns = source_columns; - temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end()); - for (const auto & joined_column : columns_added_by_join) - temp_columns.push_back(joined_column.name_and_type); - ExpressionActionsPtr temp_actions = std::make_shared(temp_columns, context); - getRootActions(func->arguments->children.at(0), true, temp_actions); + if (settings.use_index_for_in_with_subqueries) + tryMakeSetForIndexFromSubquery(arg); + } + else + { + NamesAndTypesList temp_columns = source_columns; + temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end()); + for (const auto & joined_column : columns_added_by_join) + temp_columns.push_back(joined_column.name_and_type); + ExpressionActionsPtr temp_actions = std::make_shared(temp_columns, context); + getRootActions(func->arguments->children.at(0), true, temp_actions); - Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); - if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName())) - makeExplicitSet(func, sample_block_with_calculated_columns, true, context, - settings.size_limits_for_set, prepared_sets); - } + Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); + if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName())) + makeExplicitSet(func, sample_block_with_calculated_columns, true, context, + settings.size_limits_for_set, prepared_sets); } } } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index feb8009e92e..ae698f81282 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -271,7 +271,7 @@ private: */ void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name); - void makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block); + void makeSetsForIndexImpl(const ASTPtr & node); bool isRemoteStorage() const; }; diff --git a/dbms/src/Interpreters/PreparedSets.h b/dbms/src/Interpreters/PreparedSets.h index 4276e62a239..a2c6f6de40a 100644 --- a/dbms/src/Interpreters/PreparedSets.h +++ b/dbms/src/Interpreters/PreparedSets.h @@ -1,20 +1,64 @@ #pragma once #include +#include #include #include namespace DB { -struct ASTHalfHash +struct PreparedSetKey { - UInt64 operator()(const IAST::Hash & ast_hash) const { return ast_hash.first; } + /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired + /// data types of set elements (two different Sets can be required for two tuples with the same contents + /// if left hand sides of the IN operators have different types). + static PreparedSetKey forLiteral(const IAST & ast, DataTypes types_) + { + PreparedSetKey key; + key.ast_hash = ast.getTreeHash(); + key.types = std::move(types_); + return key; + } + + /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting + /// set is fully determined by the subquery. + static PreparedSetKey forSubquery(const IAST & ast) + { + PreparedSetKey key; + key.ast_hash = ast.getTreeHash(); + return key; + } + + IAST::Hash ast_hash; + DataTypes types; /// Empty for subqueries. + + bool operator==(const PreparedSetKey & other) const + { + if (ast_hash != other.ast_hash) + return false; + + if (types.size() != other.types.size()) + return false; + + for (size_t i = 0; i < types.size(); ++i) + { + if (!types[i]->equals(*other.types[i])) + return false; + } + + return true; + } + + struct Hash + { + UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } + }; }; class Set; using SetPtr = std::shared_ptr; -using PreparedSets = std::unordered_map; +using PreparedSets = std::unordered_map; } diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index 87d9f8def3d..b64920d6233 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -12,6 +13,8 @@ #include #include #include +#include +#include namespace DB @@ -107,7 +110,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map { { "notEquals", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_NOT_IN_RANGE; out.range = Range(value); @@ -116,7 +119,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "equals", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range(value); @@ -125,7 +128,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "less", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range::createRightBounded(value, false); @@ -134,7 +137,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "greater", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range::createLeftBounded(value, false); @@ -143,7 +146,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "lessOrEquals", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range::createRightBounded(value, true); @@ -152,7 +155,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "greaterOrEquals", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range::createLeftBounded(value, true); @@ -161,25 +164,23 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "in", - [] (RPNElement & out, const Field &, const ASTPtr & node) + [] (RPNElement & out, const Field &) { out.function = RPNElement::FUNCTION_IN_SET; - out.in_function = node; return true; } }, { "notIn", - [] (RPNElement & out, const Field &, const ASTPtr & node) + [] (RPNElement & out, const Field &) { out.function = RPNElement::FUNCTION_NOT_IN_SET; - out.in_function = node; return true; } }, { "like", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { if (value.getType() != Field::Types::String) return false; @@ -458,71 +459,64 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( return found_transformation; } -void KeyCondition::getKeyTuplePositionMapping( - const ASTPtr & node, - const Context & context, - std::vector & indexes_mapping, - const size_t tuple_index, - size_t & out_key_column_num) -{ - MergeTreeSetIndex::KeyTuplePositionMapping index_mapping; - index_mapping.tuple_index = tuple_index; - DataTypePtr data_type; - if (isKeyPossiblyWrappedByMonotonicFunctions( - node, context, index_mapping.key_index, - data_type, index_mapping.functions)) - { - indexes_mapping.push_back(index_mapping); - if (out_key_column_num < index_mapping.key_index) - { - out_key_column_num = index_mapping.key_index; - } - } -} - - bool KeyCondition::tryPrepareSetIndex( - const ASTPtr & node, + const ASTs & args, const Context & context, RPNElement & out, - const SetPtr & prepared_set, size_t & out_key_column_num) { - /// The index can be prepared if the elements of the set were saved in advance. - if (!prepared_set->hasExplicitSetElements()) - return false; + const ASTPtr & left_arg = args[0]; out_key_column_num = 0; std::vector indexes_mapping; + DataTypes data_types; - size_t num_key_columns = prepared_set->getDataTypes().size(); - - const ASTFunction * node_tuple = typeid_cast(node.get()); - if (node_tuple && node_tuple->name == "tuple") + auto get_key_tuple_position_mapping = [&](const ASTPtr & node, size_t tuple_index) { - if (num_key_columns != node_tuple->arguments->children.size()) + MergeTreeSetIndex::KeyTuplePositionMapping index_mapping; + index_mapping.tuple_index = tuple_index; + DataTypePtr data_type; + if (isKeyPossiblyWrappedByMonotonicFunctions( + node, context, index_mapping.key_index, data_type, index_mapping.functions)) { - std::stringstream message; - message << "Number of columns in section IN doesn't match. " - << node_tuple->arguments->children.size() << " at left, " << num_key_columns << " at right."; - throw Exception(message.str(), ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); + indexes_mapping.push_back(index_mapping); + data_types.push_back(data_type); + if (out_key_column_num < index_mapping.key_index) + out_key_column_num = index_mapping.key_index; } + }; - size_t current_tuple_index = 0; - for (const auto & arg : node_tuple->arguments->children) - { - getKeyTuplePositionMapping(arg, context, indexes_mapping, current_tuple_index, out_key_column_num); - ++current_tuple_index; - } + const ASTFunction * left_arg_tuple = typeid_cast(left_arg.get()); + if (left_arg_tuple && left_arg_tuple->name == "tuple") + { + const auto & tuple_elements = left_arg_tuple->arguments->children; + for (size_t i = 0; i < tuple_elements.size(); ++i) + get_key_tuple_position_mapping(tuple_elements[i], i); } else - { - getKeyTuplePositionMapping(node, context, indexes_mapping, 0, out_key_column_num); - } + get_key_tuple_position_mapping(left_arg, 0); if (indexes_mapping.empty()) return false; + const ASTPtr & right_arg = args[1]; + + PreparedSetKey set_key; + if (typeid_cast(right_arg.get()) || typeid_cast(right_arg.get())) + set_key = PreparedSetKey::forSubquery(*right_arg); + else + set_key = PreparedSetKey::forLiteral(*right_arg, data_types); + + auto set_it = prepared_sets.find(set_key); + if (set_it == prepared_sets.end()) + return false; + + const SetPtr & prepared_set = set_it->second; + + /// The index can be prepared if the elements of the set were saved in advance. + if (!prepared_set->hasExplicitSetElements()) + return false; + out.set_index = std::make_shared(prepared_set->getSetElements(), std::move(indexes_mapping)); return true; @@ -640,13 +634,13 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo bool is_set_const = false; bool is_constant_transformed = false; - if (prepared_sets.count(args[1]->getTreeHash()) - && tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->getTreeHash()], key_column_num)) - { - key_arg_pos = 0; - is_set_const = true; - } - else if (getConstant(args[1], block_with_constants, const_value, const_type) + if (functionIsInOrGlobalInOperator(func->name) + && tryPrepareSetIndex(args, context, out, key_column_num)) + { + key_arg_pos = 0; + is_set_const = true; + } + else if (getConstant(args[1], block_with_constants, const_value, const_type) && isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain)) { key_arg_pos = 0; @@ -717,7 +711,7 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo if (!cast_not_needed) castValueToType(key_expr_type, const_value, const_type, node); - return atom_it->second(out, const_value, node); + return atom_it->second(out, const_value); } else if (getConstant(node, block_with_constants, const_value, const_type)) /// For cases where it says, for example, `WHERE 0 AND something` { @@ -1016,17 +1010,12 @@ bool KeyCondition::mayBeTrueInParallelogram(const std::vector & parallelo element.function == RPNElement::FUNCTION_IN_SET || element.function == RPNElement::FUNCTION_NOT_IN_SET) { - auto in_func = typeid_cast(element.in_function.get()); - const ASTs & args = typeid_cast(*in_func->arguments).children; - PreparedSets::const_iterator it = prepared_sets.find(args[1]->getTreeHash()); - if (in_func && it != prepared_sets.end()) - { - rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types)); - if (element.function == RPNElement::FUNCTION_NOT_IN_SET) - rpn_stack.back() = !rpn_stack.back(); - } - else + if (!element.set_index) throw Exception("Set for IN is not created yet", ErrorCodes::LOGICAL_ERROR); + + rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types)); + if (element.function == RPNElement::FUNCTION_NOT_IN_SET) + rpn_stack.back() = !rpn_stack.back(); } else if (element.function == RPNElement::FUNCTION_NOT) { diff --git a/dbms/src/Storages/MergeTree/KeyCondition.h b/dbms/src/Storages/MergeTree/KeyCondition.h index 1d700ad80d9..20009b50f34 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.h +++ b/dbms/src/Storages/MergeTree/KeyCondition.h @@ -310,7 +310,6 @@ private: Range range; size_t key_column = 0; /// For FUNCTION_IN_SET, FUNCTION_NOT_IN_SET - ASTPtr in_function; using MergeTreeSetIndexPtr = std::shared_ptr; MergeTreeSetIndexPtr set_index; @@ -320,7 +319,7 @@ private: using RPN = std::vector; using ColumnIndices = std::map; - using AtomMap = std::unordered_map; + using AtomMap = std::unordered_map; public: static const AtomMap atom_map; @@ -363,21 +362,13 @@ private: Field & out_value, DataTypePtr & out_type); - void getKeyTuplePositionMapping( - const ASTPtr & node, - const Context & context, - std::vector & indexes_mapping, - const size_t tuple_index, - size_t & out_key_column_num); - /// If it's possible to make an RPNElement /// that will filter values (possibly tuples) by the content of 'prepared_set', /// do it and return true. bool tryPrepareSetIndex( - const ASTPtr & node, + const ASTs & args, const Context & context, RPNElement & out, - const SetPtr & prepared_set, size_t & out_key_column_num); RPN rpn; diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 3eb9576b1c0..66a9a6f90d2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -44,7 +45,6 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( table_columns{ext::map(data.getColumns().getAllPhysical(), [] (const NameAndTypePair & col) { return col.name; })}, block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, - prepared_sets(query_info.sets), log{log} { calculateColumnSizes(data, column_names); @@ -333,8 +333,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) || (primary_key_columns.count(second_arg_name) && isConstant(args[0])) || - (primary_key_columns.count(first_arg_name) - && (typeid_cast(args[1].get()) || prepared_sets.count(args[1]->getTreeHash())))) + (primary_key_columns.count(first_arg_name) && functionIsInOrGlobalInOperator(func->name))) return true; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index ccf4970d300..5f8dd587a92 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -77,7 +77,6 @@ private: const string_set_t primary_key_columns; const string_set_t table_columns; const Block block_with_constants; - const PreparedSets & prepared_sets; Poco::Logger * log; std::unordered_map column_sizes{}; size_t total_column_size{}; From da0f5554849f7b5e9707225f5dc46aca08c6a15b Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 23 Jan 2019 16:20:03 +0300 Subject: [PATCH 021/324] speedup calculation of ASTLiteral hash a bit #4058 --- dbms/src/Parsers/ASTLiteral.cpp | 7 +++++++ dbms/src/Parsers/ASTLiteral.h | 2 ++ dbms/src/Parsers/IAST.cpp | 16 +++++++++++----- dbms/src/Parsers/IAST.h | 3 ++- 4 files changed, 22 insertions(+), 6 deletions(-) diff --git a/dbms/src/Parsers/ASTLiteral.cpp b/dbms/src/Parsers/ASTLiteral.cpp index 44c7662ba43..04504fb3594 100644 --- a/dbms/src/Parsers/ASTLiteral.cpp +++ b/dbms/src/Parsers/ASTLiteral.cpp @@ -7,6 +7,13 @@ namespace DB { +void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const +{ + const char * prefix = "Literal_"; + hash_state.update(prefix, strlen(prefix)); + applyVisitor(FieldVisitorHash(hash_state), value); +} + void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const { /// Special case for very large arrays. Instead of listing all elements, will use hash of them. diff --git a/dbms/src/Parsers/ASTLiteral.h b/dbms/src/Parsers/ASTLiteral.h index d6b8b4efc3b..dd5bb572e7d 100644 --- a/dbms/src/Parsers/ASTLiteral.h +++ b/dbms/src/Parsers/ASTLiteral.h @@ -22,6 +22,8 @@ public: ASTPtr clone() const override { return std::make_shared(*this); } + void updateTreeHashImpl(SipHash & hash_state) const override; + protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override { diff --git a/dbms/src/Parsers/IAST.cpp b/dbms/src/Parsers/IAST.cpp index a3244e1abeb..74956c43166 100644 --- a/dbms/src/Parsers/IAST.cpp +++ b/dbms/src/Parsers/IAST.cpp @@ -53,20 +53,26 @@ size_t IAST::checkSize(size_t max_size) const IAST::Hash IAST::getTreeHash() const { SipHash hash_state; - getTreeHashImpl(hash_state); + updateTreeHash(hash_state); IAST::Hash res; hash_state.get128(res.first, res.second); return res; } -void IAST::getTreeHashImpl(SipHash & hash_state) const +void IAST::updateTreeHash(SipHash & hash_state) const +{ + updateTreeHashImpl(hash_state); + hash_state.update(children.size()); + for (const auto & child : children) + child->updateTreeHash(hash_state); +} + + +void IAST::updateTreeHashImpl(SipHash & hash_state) const { auto id = getID(); hash_state.update(id.data(), id.size()); - hash_state.update(children.size()); - for (const auto & child : children) - child->getTreeHashImpl(hash_state); } diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index 563eeb3c125..7692691073d 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -76,7 +76,8 @@ public: */ using Hash = std::pair; Hash getTreeHash() const; - void getTreeHashImpl(SipHash & hash_state) const; + void updateTreeHash(SipHash & hash_state) const; + virtual void updateTreeHashImpl(SipHash & hash_state) const; void dumpTree(std::ostream & ostr, size_t indent = 0) const { From b623d4302c9d86d21ecc4388d3114f1e8040fab9 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 23 Jan 2019 16:27:01 +0300 Subject: [PATCH 022/324] add clone() calls when copying ASTs from stored mutation commands (just in case) #4058 --- dbms/src/Interpreters/MutationsInterpreter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 64f30bdbc8a..01db6679e6b 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -53,7 +53,7 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const select->select_expression_list->children.push_back(count_func); if (commands.size() == 1) - select->where_expression = commands[0].predicate; + select->where_expression = commands[0].predicate->clone(); else { auto coalesced_predicates = std::make_shared(); @@ -62,7 +62,7 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const coalesced_predicates->children.push_back(coalesced_predicates->arguments); for (const MutationCommand & command : commands) - coalesced_predicates->arguments->children.push_back(command.predicate); + coalesced_predicates->arguments->children.push_back(command.predicate->clone()); select->where_expression = std::move(coalesced_predicates); } From 8ad1a55f3bfe1015f41fb76aecc98df221d08b08 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Tue, 22 Jan 2019 17:47:43 +0100 Subject: [PATCH 023/324] implement sumMapFiltered --- .../AggregateFunctionSumMap.cpp | 35 ++++++++++-- .../AggregateFunctionSumMap.h | 54 +++++++++++++++++-- .../0_stateless/00502_sum_map.reference | 2 + .../queries/0_stateless/00502_sum_map.sql | 3 ++ .../agg_functions/parametric_functions.md | 5 ++ 5 files changed, 92 insertions(+), 7 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 571d6f5c0a1..5138d8f1f02 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -12,10 +12,10 @@ namespace DB namespace { -AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, const DataTypes & arguments, const Array & params) -{ - assertNoParameters(name, params); +using SumMapArgs = std::pair; +SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) +{ if (arguments.size() < 2) throw Exception("Aggregate function " + name + " requires at least two arguments of Array type.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -25,6 +25,7 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con throw Exception("First argument for function " + name + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + const DataTypePtr & keys_type = array_type->getNestedType(); DataTypes values_types; @@ -37,6 +38,15 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con values_types.push_back(array_type->getNestedType()); } + return {keys_type, std::move(values_types)}; +} + +AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, const DataTypes & arguments, const Array & params) +{ + assertNoParameters(name, params); + + auto [keys_type, values_types] = parseArguments(name, arguments); + AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types)); if (!res) res.reset(createWithDecimalType(*keys_type, keys_type, values_types)); @@ -46,11 +56,30 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con return res; } +AggregateFunctionPtr createAggregateFunctionSumMapFiltered(const std::string & name, const DataTypes & arguments, const Array & params) +{ + if (params.size() != 1) + throw Exception("Aggregate function " + name + "requires exactly one parameter of Array type.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + Array keys_to_keep = params.front().safeGet(); + + auto [keys_type, values_types] = parseArguments(name, arguments); + + AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types, keys_to_keep)); + if (!res) + res.reset(createWithDecimalType(*keys_type, keys_type, values_types, keys_to_keep)); + if (!res) + throw Exception("Illegal type of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return res; +} } void registerAggregateFunctionSumMap(AggregateFunctionFactory & factory) { factory.registerFunction("sumMap", createAggregateFunctionSumMap); + factory.registerFunction("sumMapFiltered", createAggregateFunctionSumMapFiltered); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 4a20a314789..e9c70eaa5f1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -50,9 +50,9 @@ struct AggregateFunctionSumMapData * ([1,2,3,4,5,6,7,8,9,10],[10,10,45,20,35,20,15,30,20,20]) */ -template -class AggregateFunctionSumMap final : public IAggregateFunctionDataHelper< - AggregateFunctionSumMapData>, AggregateFunctionSumMap> +template +class AggregateFunctionSumMapBase : public IAggregateFunctionDataHelper< + AggregateFunctionSumMapData>, Derived> { private: using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; @@ -61,7 +61,7 @@ private: DataTypes values_types; public: - AggregateFunctionSumMap(const DataTypePtr & keys_type, const DataTypes & values_types) + AggregateFunctionSumMapBase(const DataTypePtr & keys_type, const DataTypes & values_types) : keys_type(keys_type), values_types(values_types) {} String getName() const override { return "sumMap"; } @@ -109,6 +109,11 @@ public: array_column.getData().get(values_vec_offset + i, value); const auto & key = keys_vec.getData()[keys_vec_offset + i]; + if (!keepKey(key)) + { + continue; + } + IteratorType it; if constexpr (IsDecimalNumber) { @@ -253,6 +258,47 @@ public: } const char * getHeaderFilePath() const override { return __FILE__; } + + virtual bool keepKey(const T & key) const = 0; +}; + +template +class AggregateFunctionSumMap final : public AggregateFunctionSumMapBase> +{ +public: + AggregateFunctionSumMap(const DataTypePtr & keys_type, DataTypes & values_types) + : AggregateFunctionSumMapBase>{keys_type, values_types} + {} + + String getName() const override { return "sumMap"; } + + bool keepKey(const T &) const override { return true; } +}; + +template +class AggregateFunctionSumMapFiltered final : public AggregateFunctionSumMapBase> +{ +private: + std::vector keys_to_keep; + +public: + AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep) + : AggregateFunctionSumMapBase>{keys_type, values_types} + { + this->keys_to_keep.reserve(keys_to_keep.size()); + for (const Field & f : keys_to_keep) + { + this->keys_to_keep.emplace_back(f.safeGet>()); + } + std::sort(begin(this->keys_to_keep), end(this->keys_to_keep)); + } + + String getName() const override { return "sumMapFiltered"; } + + bool keepKey(const T & key) const override + { + return std::binary_search(begin(keys_to_keep), end(keys_to_keep), key); + } }; } diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.reference b/dbms/tests/queries/0_stateless/00502_sum_map.reference index 6da96805974..ac5678ebeab 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.reference +++ b/dbms/tests/queries/0_stateless/00502_sum_map.reference @@ -8,6 +8,8 @@ 2000-01-01 00:01:00 ([4,5,6,7,8],[10,10,20,10,10]) 2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10] 2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10] +([1],[10]) +([1, 4, 8], [10, 20, 10]) ([1],[1]) ([1],[1]) (['a'],[1]) diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.sql b/dbms/tests/queries/0_stateless/00502_sum_map.sql index e6377155dac..9cf941dd908 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.sql +++ b/dbms/tests/queries/0_stateless/00502_sum_map.sql @@ -12,6 +12,9 @@ SELECT sumMapMerge(s) FROM (SELECT sumMapState(statusMap.status, statusMap.reque SELECT timeslot, sumMap(statusMap.status, statusMap.requests) FROM test.sum_map GROUP BY timeslot ORDER BY timeslot; SELECT timeslot, sumMap(statusMap.status, statusMap.requests).1, sumMap(statusMap.status, statusMap.requests).2 FROM test.sum_map GROUP BY timeslot ORDER BY timeslot; +SELECT sumMapFiltered([1])(statusMap.status, statusMap.requests) FROM test.sum_map; +SELECT sumMapFiltered([1, 4, 8])(statusMap.status, statusMap.requests) FROM test.sum_map; + DROP TABLE test.sum_map; select sumMap(val, cnt) from ( SELECT [ CAST(1, 'UInt64') ] as val, [1] as cnt ); diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index 1cbe784e621..15b9c3360fa 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -155,3 +155,8 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= ``` [Original article](https://clickhouse.yandex/docs/en/query_language/agg_functions/parametric_functions/) + + +## sumMapFiltered(keys_to_keep)(keys, values) + +Same behavior as [sumMap](reference.md#sumMap) except that an array of keys is passed as a parameter. This can be especially useful when working with a high cardinality of keys. From 0ac0557e0f13c1049be42916da190d4eea0b513e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:19:23 +0300 Subject: [PATCH 024/324] Make DBMS dependent on the protobuf library. --- CMakeLists.txt | 1 + contrib/CMakeLists.txt | 14 +++++++------- dbms/CMakeLists.txt | 3 +++ 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4265cc126f1..6bd0aae5fd1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -232,6 +232,7 @@ include (cmake/find_llvm.cmake) include (cmake/find_cpuid.cmake) include (cmake/find_libgsasl.cmake) include (cmake/find_libxml2.cmake) +include (cmake/find_protobuf.cmake) include (cmake/find_hdfs3.cmake) include (cmake/find_consistent-hashing.cmake) include (cmake/find_base64.cmake) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 7b66bfce063..0c4b6c15287 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -207,14 +207,14 @@ if (USE_INTERNAL_LIBXML2_LIBRARY) add_subdirectory(libxml2-cmake) endif () +if (USE_INTERNAL_PROTOBUF_LIBRARY) + set(protobuf_BUILD_TESTS OFF CACHE INTERNAL "" FORCE) + set(protobuf_BUILD_SHARED_LIBS OFF CACHE INTERNAL "" FORCE) + set(protobuf_WITH_ZLIB 0 CACHE INTERNAL "" FORCE) # actually will use zlib, but skip find + add_subdirectory(protobuf/cmake) +endif () + if (USE_INTERNAL_HDFS3_LIBRARY) - include(${ClickHouse_SOURCE_DIR}/cmake/find_protobuf.cmake) - if (USE_INTERNAL_PROTOBUF_LIBRARY) - set(protobuf_BUILD_TESTS OFF CACHE INTERNAL "" FORCE) - set(protobuf_BUILD_SHARED_LIBS OFF CACHE INTERNAL "" FORCE) - set(protobuf_WITH_ZLIB 0 CACHE INTERNAL "" FORCE) # actually will use zlib, but skip find - add_subdirectory(protobuf/cmake) - endif () add_subdirectory(libhdfs3-cmake) endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 3ce29e9d65e..cd539444c97 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -298,6 +298,9 @@ target_link_libraries(dbms PRIVATE ${OPENSSL_CRYPTO_LIBRARY} Threads::Threads) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${DIVIDE_INCLUDE_DIR}) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) +target_link_libraries (dbms PRIVATE ${Protobuf_LIBRARY}) +target_include_directories (dbms SYSTEM BEFORE PRIVATE ${Protobuf_INCLUDE_DIR}) + if (USE_HDFS) target_link_libraries (clickhouse_common_io PRIVATE ${HDFS3_LIBRARY}) target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${HDFS3_INCLUDE_DIR}) From 67fca7e02929ba9f4939657179beea5bae8237c4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:23:37 +0300 Subject: [PATCH 025/324] Add support for settings 'format_schema' in an output query. This fixes syntax error when executing 'SELECT ... FORMAT ... SETTINGS format_schema=...' --- dbms/programs/client/Client.cpp | 5 +++++ dbms/src/Interpreters/executeQuery.cpp | 4 ++++ dbms/src/Parsers/ASTQueryWithOutput.cpp | 12 ++++++++++++ dbms/src/Parsers/ASTQueryWithOutput.h | 4 +++- dbms/src/Parsers/ParserQueryWithOutput.cpp | 11 +++++++++++ dbms/src/Parsers/ParserQueryWithOutput.h | 2 +- 6 files changed, 36 insertions(+), 2 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 0edffbcf4de..66acc7fcea5 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -56,6 +56,7 @@ #include #include #include +#include #include #include #include @@ -1206,6 +1207,10 @@ private: const auto & id = typeid_cast(*query_with_output->format); current_format = id.name; } + if (query_with_output->settings) + { + InterpreterSetQuery(query_with_output->settings, context).executeForCurrentContext(); + } } if (has_vertical_output_suffix) diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 3911e437fa6..435de7c6f1f 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include "DNSCacheUpdater.h" @@ -502,6 +503,9 @@ void executeQuery( ? *getIdentifierName(ast_query_with_output->format) : context.getDefaultFormat(); + if (ast_query_with_output && ast_query_with_output->settings) + InterpreterSetQuery(ast_query_with_output->settings, context).executeForCurrentContext(); + BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader()); if (auto stream = dynamic_cast(streams.in.get())) diff --git a/dbms/src/Parsers/ASTQueryWithOutput.cpp b/dbms/src/Parsers/ASTQueryWithOutput.cpp index 62abe5de9e1..fd6957c436b 100644 --- a/dbms/src/Parsers/ASTQueryWithOutput.cpp +++ b/dbms/src/Parsers/ASTQueryWithOutput.cpp @@ -15,6 +15,11 @@ void ASTQueryWithOutput::cloneOutputOptions(ASTQueryWithOutput & cloned) const cloned.format = format->clone(); cloned.children.push_back(cloned.format); } + if (settings) + { + cloned.settings = settings->clone(); + cloned.children.push_back(cloned.settings); + } } void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const @@ -34,6 +39,12 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FORMAT " << (s.hilite ? hilite_none : ""); format->formatImpl(s, state, frame); } + + if (settings) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SETTINGS " << (s.hilite ? hilite_none : ""); + settings->formatImpl(s, state, frame); + } } bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast) @@ -42,6 +53,7 @@ bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast) { ast_with_output->format.reset(); ast_with_output->out_file.reset(); + ast_with_output->settings.reset(); return true; } diff --git a/dbms/src/Parsers/ASTQueryWithOutput.h b/dbms/src/Parsers/ASTQueryWithOutput.h index 0660b1bec63..ec525b3aca9 100644 --- a/dbms/src/Parsers/ASTQueryWithOutput.h +++ b/dbms/src/Parsers/ASTQueryWithOutput.h @@ -6,13 +6,15 @@ namespace DB { -/** Query with output options (supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] suffix). +/** Query with output options + * (supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] [SETTINGS key1 = value1, key2 = value2, ...] suffix). */ class ASTQueryWithOutput : public IAST { public: ASTPtr out_file; ASTPtr format; + ASTPtr settings; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const final; diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index 2e1a5ff529a..830060cb953 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -81,6 +82,16 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query_with_output.children.push_back(query_with_output.format); } + // SETTINGS key1 = value1, key2 = value2, ... + ParserKeyword s_settings("SETTINGS"); + if (s_settings.ignore(pos, expected)) + { + ParserSetQuery parser_settings(true); + if (!parser_settings.parse(pos, query_with_output.settings, expected)) + return false; + query_with_output.children.push_back(query_with_output.settings); + } + if (explain_ast) { node = std::make_shared(); diff --git a/dbms/src/Parsers/ParserQueryWithOutput.h b/dbms/src/Parsers/ParserQueryWithOutput.h index b0d08465354..3163bc38437 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.h +++ b/dbms/src/Parsers/ParserQueryWithOutput.h @@ -7,7 +7,7 @@ namespace DB { -/// Parse queries supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] suffix. +/// Parse queries supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] [SETTINGS key1 = value1, key2 = value2, ...] suffix. class ParserQueryWithOutput : public IParserBase { public: From efb1cb0528601369f3ca89a95fff8ff999809588 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 08:58:57 +0100 Subject: [PATCH 026/324] sumMapFiltered: remove `this->` in constructor --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index e9c70eaa5f1..1e39db916f0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -282,15 +282,15 @@ private: std::vector keys_to_keep; public: - AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep) + AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep_) : AggregateFunctionSumMapBase>{keys_type, values_types} { - this->keys_to_keep.reserve(keys_to_keep.size()); - for (const Field & f : keys_to_keep) + keys_to_keep.reserve(keys_to_keep_.size()); + for (const Field & f : keys_to_keep_) { - this->keys_to_keep.emplace_back(f.safeGet>()); + keys_to_keep.emplace_back(f.safeGet>()); } - std::sort(begin(this->keys_to_keep), end(this->keys_to_keep)); + std::sort(begin(keys_to_keep), end(keys_to_keep)); } String getName() const override { return "sumMapFiltered"; } From 22f7bd03b9bb03a6d83317c16cff4d83d67a0ea7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 09:02:17 +0100 Subject: [PATCH 027/324] sumMapFiltered: use CRTP instead of virtual for keepKey --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 1e39db916f0..17dcf91d5e1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -259,7 +259,7 @@ public: const char * getHeaderFilePath() const override { return __FILE__; } - virtual bool keepKey(const T & key) const = 0; + bool keepKey(const T & key) const { return static_cast(*this).keepKey(key); } }; template @@ -272,7 +272,7 @@ public: String getName() const override { return "sumMap"; } - bool keepKey(const T &) const override { return true; } + bool keepKey(const T &) const { return true; } }; template @@ -295,7 +295,7 @@ public: String getName() const override { return "sumMapFiltered"; } - bool keepKey(const T & key) const override + bool keepKey(const T & key) const { return std::binary_search(begin(keys_to_keep), end(keys_to_keep), key); } From a8647761ce462a54fb007377834e5f579986825b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 09:10:08 +0100 Subject: [PATCH 028/324] sumMapFiltered: fix dangling reference --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 5138d8f1f02..151e64640eb 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -12,7 +12,7 @@ namespace DB namespace { -using SumMapArgs = std::pair; +using SumMapArgs = std::pair; SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) { @@ -26,9 +26,10 @@ SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - const DataTypePtr & keys_type = array_type->getNestedType(); + DataTypePtr keys_type = array_type->getNestedType(); DataTypes values_types; + values_types.reserve(arguments.size() - 1); for (size_t i = 1; i < arguments.size(); ++i) { array_type = checkAndGetDataType(arguments[i].get()); @@ -38,7 +39,7 @@ SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) values_types.push_back(array_type->getNestedType()); } - return {keys_type, std::move(values_types)}; + return {std::move(keys_type), std::move(values_types)}; } AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, const DataTypes & arguments, const Array & params) From 90afad8c00a523267271c6fa77cc599f91522bef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 13:30:31 +0100 Subject: [PATCH 029/324] sumMapFiltered: improve error reporting on parameter type mismatch --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 151e64640eb..02303b953d9 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -60,10 +60,13 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con AggregateFunctionPtr createAggregateFunctionSumMapFiltered(const std::string & name, const DataTypes & arguments, const Array & params) { if (params.size() != 1) - throw Exception("Aggregate function " + name + "requires exactly one parameter of Array type.", + throw Exception("Aggregate function " + name + " requires exactly one parameter of Array type.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - Array keys_to_keep = params.front().safeGet(); + Array keys_to_keep; + if (!params.front().tryGet(keys_to_keep)) + throw Exception("Aggregate function " + name + " requires an Array as parameter.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto [keys_type, values_types] = parseArguments(name, arguments); From ba1f1c74608f766d60493b8e928930ad43d0f965 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 13:31:33 +0100 Subject: [PATCH 030/324] sumMapFiltered: use an unordered_set --- .../AggregateFunctionSumMap.h | 10 +++------- dbms/src/Core/Types.h | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 17dcf91d5e1..1e5f3e38cd2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -279,7 +279,7 @@ template class AggregateFunctionSumMapFiltered final : public AggregateFunctionSumMapBase> { private: - std::vector keys_to_keep; + std::unordered_set keys_to_keep; public: AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep_) @@ -288,17 +288,13 @@ public: keys_to_keep.reserve(keys_to_keep_.size()); for (const Field & f : keys_to_keep_) { - keys_to_keep.emplace_back(f.safeGet>()); + keys_to_keep.emplace(f.safeGet>()); } - std::sort(begin(keys_to_keep), end(keys_to_keep)); } String getName() const override { return "sumMapFiltered"; } - bool keepKey(const T & key) const - { - return std::binary_search(begin(keys_to_keep), end(keys_to_keep), key); - } + bool keepKey(const T & key) const { return keys_to_keep.count(key); } }; } diff --git a/dbms/src/Core/Types.h b/dbms/src/Core/Types.h index 5e2cd47f440..e4882cd64f7 100644 --- a/dbms/src/Core/Types.h +++ b/dbms/src/Core/Types.h @@ -166,3 +166,20 @@ template <> constexpr bool IsDecimalNumber = true; template <> constexpr bool IsDecimalNumber = true; } + +/// Specialization of `std::hash` for the Decimal types. +namespace std +{ + template + struct hash> { size_t operator()(const DB::Decimal & x) const { return hash()(x.value); } }; + + template <> + struct hash + { + size_t operator()(const DB::Decimal128 & x) const + { + return std::hash()(x.value >> 64) + ^ std::hash()(x.value & std::numeric_limits::max()); + } + }; +} From e64dc4798e23a3d335ad07f69bbe9b2c7ae18e9b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 14:54:29 +0100 Subject: [PATCH 031/324] sumMapFiltered: fix test output --- dbms/tests/queries/0_stateless/00502_sum_map.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.reference b/dbms/tests/queries/0_stateless/00502_sum_map.reference index ac5678ebeab..67639ad230f 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.reference +++ b/dbms/tests/queries/0_stateless/00502_sum_map.reference @@ -9,7 +9,7 @@ 2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10] 2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10] ([1],[10]) -([1, 4, 8], [10, 20, 10]) +([1,4,8], [10,20,10]) ([1],[1]) ([1],[1]) (['a'],[1]) From cbbf04204b22c93231fe0fbb8c920b2644ee628b Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 24 Jan 2019 17:22:58 +0300 Subject: [PATCH 032/324] JOIN with USING: do not duplicate columns --- dbms/src/Interpreters/QueryNormalizer.cpp | 88 +++++++++++++++++-- dbms/src/Interpreters/QueryNormalizer.h | 28 +++++- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 41 ++------- .../00051_any_inner_join.reference | 10 +-- .../0_stateless/00052_all_left_join.reference | 30 +++---- .../00053_all_inner_join.reference | 20 ++--- .../0_stateless/00054_join_string.reference | 30 +++---- .../00555_right_join_excessive_rows.sql | 2 +- ...query_aggregation_column_removal.reference | 8 +- .../00597_push_down_predicate.reference | 8 +- .../00679_replace_asterisk.reference | 6 +- .../00702_join_with_using.reference | 42 ++++++++- .../0_stateless/00702_join_with_using.sql | 26 ++++++ .../0_stateless/00725_join_on_bug_2.sql | 2 +- .../00740_database_in_nested_view.sql | 2 +- .../00800_low_cardinality_join.reference | 18 ++-- 16 files changed, 245 insertions(+), 116 deletions(-) diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index b8f11ddbf79..aa00e6e8b5c 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -1,13 +1,16 @@ +#include #include #include +#include +#include #include #include #include +#include #include #include #include #include -#include #include #include @@ -16,10 +19,14 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int TOO_DEEP_AST; extern const int CYCLIC_ALIASES; } +NameSet removeDuplicateColumns(NamesAndTypesList & columns); + + class CheckASTDepth { public: @@ -135,7 +142,8 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// Replace *, alias.*, database.table.* with a list of columns. void QueryNormalizer::visit(ASTExpressionList & node, const ASTPtr &, Data & data) { - auto & tables_with_columns = data.tables_with_columns; + const auto & tables_with_columns = data.tables_with_columns; + const auto & source_columns_set = data.source_columns_set; ASTs old_children; if (data.processAsterisks()) @@ -162,22 +170,43 @@ void QueryNormalizer::visit(ASTExpressionList & node, const ASTPtr &, Data & dat { if (typeid_cast(child.get())) { - for (const auto & pr : tables_with_columns) - for (const auto & column_name : pr.second) - node.children.emplace_back(std::make_shared(column_name)); + bool first_table = true; + for (const auto & [table_name, table_columns] : tables_with_columns) + { + for (const auto & column_name : table_columns) + if (first_table || !data.join_using_columns.count(column_name)) + { + /// qualifed names for duplicates + if (!first_table && source_columns_set && source_columns_set->count(column_name)) + node.children.emplace_back(std::make_shared(table_name.getQualifiedNamePrefix() + column_name)); + else + node.children.emplace_back(std::make_shared(column_name)); + } + + first_table = false; + } } else if (const auto * qualified_asterisk = typeid_cast(child.get())) { DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]); + bool first_table = true; for (const auto & [table_name, table_columns] : tables_with_columns) { if (ident_db_and_name.satisfies(table_name, true)) { for (const auto & column_name : table_columns) - node.children.emplace_back(std::make_shared(column_name)); + { + /// qualifed names for duplicates + if (!first_table && source_columns_set && source_columns_set->count(column_name)) + node.children.emplace_back(std::make_shared(table_name.getQualifiedNamePrefix() + column_name)); + else + node.children.emplace_back(std::make_shared(column_name)); + } break; } + + first_table = false; } } else @@ -198,6 +227,11 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr & /// special visitChildren() for ASTSelectQuery void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr & ast, Data & data) { + extractTablesWithColumns(select, data); + + if (auto join = select.join()) + extractJoinUsingColumns(join->table_join, data); + for (auto & child : ast->children) { if (typeid_cast(child.get()) || @@ -312,4 +346,46 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data) } } +void QueryNormalizer::extractTablesWithColumns(const ASTSelectQuery & select_query, Data & data) +{ + if (data.context && select_query.tables && !select_query.tables->children.empty()) + { + data.tables_with_columns.clear(); + String current_database = data.context->getCurrentDatabase(); + + for (const ASTTableExpression * table_expression : getSelectTablesExpression(select_query)) + { + DatabaseAndTableWithAlias table_name(*table_expression, current_database); + + NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, *data.context); + removeDuplicateColumns(names_and_types); + + data.tables_with_columns.emplace_back(std::move(table_name), names_and_types.getNames()); + } + } +} + +/// 'select * from a join b using id' should result one 'id' column +void QueryNormalizer::extractJoinUsingColumns(const ASTPtr ast, Data & data) +{ + const auto & table_join = typeid_cast(*ast); + + if (table_join.using_expression_list) + { + auto & keys = typeid_cast(*table_join.using_expression_list); + for (const auto & key : keys.children) + if (auto opt_column = getIdentifierName(key)) + data.join_using_columns.insert(*opt_column); + else if (auto * literal = typeid_cast(key.get())) + data.join_using_columns.insert(key->getColumnName()); + else + { + String alias = key->tryGetAlias(); + if (alias.empty()) + throw Exception("Logical error: expected identifier or alias, got: " + key->getID(), ErrorCodes::LOGICAL_ERROR); + data.join_using_columns.insert(alias); + } + } +} + } diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 3e55e0253e6..4c72328a92d 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -17,11 +19,11 @@ inline bool functionIsInOrGlobalInOperator(const String & name) return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn"; } - class ASTFunction; class ASTIdentifier; class ASTExpressionList; struct ASTTablesInSelectQueryElement; +class Context; class QueryNormalizer @@ -52,7 +54,10 @@ public: const Aliases & aliases; const ExtractedSettings settings; - const std::vector tables_with_columns; + const Context * context; + const NameSet * source_columns_set; + std::vector tables_with_columns; + std::unordered_set join_using_columns; /// tmp data size_t level; @@ -60,10 +65,22 @@ public: SetOfASTs current_asts; /// vertices in the current call stack of this method std::string current_alias; /// the alias referencing to the ancestor of ast (the deepest ancestor with aliases) - Data(const Aliases & aliases_, ExtractedSettings && settings_, std::vector && tables_with_columns_ = {}) + Data(const Aliases & aliases_, ExtractedSettings && settings_, const Context & context_, + const NameSet & source_columns_set, Names && all_columns) : aliases(aliases_) , settings(settings_) - , tables_with_columns(tables_with_columns_) + , context(&context_) + , source_columns_set(&source_columns_set) + , level(0) + { + tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(all_columns)); + } + + Data(const Aliases & aliases_, ExtractedSettings && settings_) + : aliases(aliases_) + , settings(settings_) + , context(nullptr) + , source_columns_set(nullptr) , level(0) {} @@ -91,6 +108,9 @@ private: static void visit(ASTSelectQuery &, const ASTPtr &, Data &); static void visitChildren(const ASTPtr &, Data & data); + + static void extractTablesWithColumns(const ASTSelectQuery & select_query, Data & data); + static void extractJoinUsingColumns(const ASTPtr ast, Data & data); }; } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index e09f1b3a95b..fb6a34d37b9 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -100,12 +100,13 @@ void normalizeTree( const Names & source_columns, const NameSet & source_columns_set, const Context & context, - const ASTSelectQuery * select_query, - bool asterisk_left_columns_only) + const ASTSelectQuery * select_query) { + const auto & settings = context.getSettingsRef(); + Names all_columns_name = source_columns; - if (!asterisk_left_columns_only) + if (!settings.asterisk_left_columns_only) { auto columns_from_joined_table = result.analyzed_join.getColumnsFromJoinedTable(source_columns_set, context, select_query); for (auto & column : columns_from_joined_table) @@ -115,37 +116,7 @@ void normalizeTree( if (all_columns_name.empty()) throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR); - std::vector table_with_columns; - if (select_query && select_query->tables && !select_query->tables->children.empty()) - { - std::vector tables_expression = getSelectTablesExpression(*select_query); - - bool first = true; - String current_database = context.getCurrentDatabase(); - for (const auto * table_expression : tables_expression) - { - DatabaseAndTableWithAlias table_name(*table_expression, current_database); - NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context); - - removeDuplicateColumns(names_and_types); - - if (!first) - { - /// For joined tables qualify duplicating names. - for (auto & name_and_type : names_and_types) - if (source_columns_set.count(name_and_type.name)) - name_and_type.name = table_name.getQualifiedNamePrefix() + name_and_type.name; - } - - first = false; - - table_with_columns.emplace_back(std::move(table_name), names_and_types.getNames()); - } - } - else - table_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(all_columns_name)); - - QueryNormalizer::Data normalizer_data(result.aliases, context.getSettingsRef(), std::move(table_with_columns)); + QueryNormalizer::Data normalizer_data(result.aliases, settings, context, source_columns_set, std::move(all_columns_name)); QueryNormalizer(normalizer_data).visit(query); } @@ -754,7 +725,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Common subexpression elimination. Rewrite rules. normalizeTree(query, result, (storage ? storage->getColumns().ordinary.getNames() : source_columns_list), source_columns_set, - context, select_query, settings.asterisk_left_columns_only != 0); + context, select_query); /// Remove unneeded columns according to 'required_result_columns'. /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. diff --git a/dbms/tests/queries/0_stateless/00051_any_inner_join.reference b/dbms/tests/queries/0_stateless/00051_any_inner_join.reference index a7949c6a350..2a61b78f6ea 100644 --- a/dbms/tests/queries/0_stateless/00051_any_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00051_any_inner_join.reference @@ -1,5 +1,5 @@ -0 0 0 -2 2 1 -4 4 2 -6 6 3 -8 8 4 +0 0 +2 1 +4 2 +6 3 +8 4 diff --git a/dbms/tests/queries/0_stateless/00052_all_left_join.reference b/dbms/tests/queries/0_stateless/00052_all_left_join.reference index 7d48d304488..a33465fcd1d 100644 --- a/dbms/tests/queries/0_stateless/00052_all_left_join.reference +++ b/dbms/tests/queries/0_stateless/00052_all_left_join.reference @@ -1,15 +1,15 @@ -0 0 0 -0 0 1 -1 1 2 -1 1 3 -2 2 4 -2 2 5 -3 3 6 -3 3 7 -4 4 8 -4 4 9 -5 5 0 -6 6 0 -7 7 0 -8 8 0 -9 9 0 +0 0 +0 1 +1 2 +1 3 +2 4 +2 5 +3 6 +3 7 +4 8 +4 9 +5 0 +6 0 +7 0 +8 0 +9 0 diff --git a/dbms/tests/queries/0_stateless/00053_all_inner_join.reference b/dbms/tests/queries/0_stateless/00053_all_inner_join.reference index 24857668974..41707378e8d 100644 --- a/dbms/tests/queries/0_stateless/00053_all_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00053_all_inner_join.reference @@ -1,10 +1,10 @@ -0 0 0 -0 0 1 -1 1 2 -1 1 3 -2 2 4 -2 2 5 -3 3 6 -3 3 7 -4 4 8 -4 4 9 +0 0 +0 1 +1 2 +1 3 +2 4 +2 5 +3 6 +3 7 +4 8 +4 9 diff --git a/dbms/tests/queries/0_stateless/00054_join_string.reference b/dbms/tests/queries/0_stateless/00054_join_string.reference index 4c35b240b32..75a0a5bc5fc 100644 --- a/dbms/tests/queries/0_stateless/00054_join_string.reference +++ b/dbms/tests/queries/0_stateless/00054_join_string.reference @@ -1,15 +1,15 @@ -A A 0 -A A 1 -B B 2 -B B 3 -C C 4 -C C 5 -D D 6 -D D 7 -E E 8 -E E 9 -F F 0 -G G 0 -H H 0 -I I 0 -J J 0 +A 0 +A 1 +B 2 +B 3 +C 4 +C 5 +D 6 +D 7 +E 8 +E 9 +F 0 +G 0 +H 0 +I 0 +J 0 diff --git a/dbms/tests/queries/0_stateless/00555_right_join_excessive_rows.sql b/dbms/tests/queries/0_stateless/00555_right_join_excessive_rows.sql index 668ad5e1bc9..1ec82d5f1b4 100644 --- a/dbms/tests/queries/0_stateless/00555_right_join_excessive_rows.sql +++ b/dbms/tests/queries/0_stateless/00555_right_join_excessive_rows.sql @@ -1,2 +1,2 @@ SET max_block_size = 10; -SELECT * FROM (select toUInt64(1) s limit 1) any right join (select number s from numbers(11)) using (s) ORDER BY s; +SELECT * FROM (select toUInt64(1) s limit 1) any right join (select number s, s as x from numbers(11)) using (s) ORDER BY s; diff --git a/dbms/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.reference b/dbms/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.reference index 936106d559e..a0265bdb7ed 100644 --- a/dbms/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.reference +++ b/dbms/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.reference @@ -14,10 +14,10 @@ 2 facebook.com 1 google.com 2 yandex.ru -1 baidu.com 1 baidu.com -1 google.com 1 google.com -2 facebook.com 2 facebook.com -2 yandex.ru 2 yandex.ru +1 baidu.com +1 google.com +2 facebook.com +2 yandex.ru 1 1 2 diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index 3dd49a0bede..df5aebabc89 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -3,7 +3,7 @@ 1 1 -------Need push down------- -0 0 +0 1 1 1 @@ -23,13 +23,13 @@ 1 2000-01-01 1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 -2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 +2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 2000-01-01 1 test string 1 1 -1 2000-01-01 2000-01-01 1 test string 1 1 +1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 2000-01-01 2 test string 2 2 1 1 -------Push to having expression, need check.------- -------Compatibility test------- -1 2000-01-01 2000-01-01 1 test string 1 1 +1 2000-01-01 test string 1 1 diff --git a/dbms/tests/queries/0_stateless/00679_replace_asterisk.reference b/dbms/tests/queries/0_stateless/00679_replace_asterisk.reference index ab22461f6f2..d05e3183147 100644 --- a/dbms/tests/queries/0_stateless/00679_replace_asterisk.reference +++ b/dbms/tests/queries/0_stateless/00679_replace_asterisk.reference @@ -1,4 +1,4 @@ 1 2 -1 2 3 1 4 5 -1 2 1 3 1 3 -1 2 1 3 1 3 3 +1 2 3 4 5 +1 2 3 1 3 +1 2 3 1 3 3 diff --git a/dbms/tests/queries/0_stateless/00702_join_with_using.reference b/dbms/tests/queries/0_stateless/00702_join_with_using.reference index 53594f8d906..c9035acc656 100644 --- a/dbms/tests/queries/0_stateless/00702_join_with_using.reference +++ b/dbms/tests/queries/0_stateless/00702_join_with_using.reference @@ -1,3 +1,39 @@ -1 1 1 1 -2 2 2 2 -3 3 3 3 +1 1 +2 2 +3 3 +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert diff --git a/dbms/tests/queries/0_stateless/00702_join_with_using.sql b/dbms/tests/queries/0_stateless/00702_join_with_using.sql index 4dcb4461018..6956730eabf 100644 --- a/dbms/tests/queries/0_stateless/00702_join_with_using.sql +++ b/dbms/tests/queries/0_stateless/00702_join_with_using.sql @@ -11,3 +11,29 @@ SELECT * FROM test.using1 ALL LEFT JOIN (SELECT * FROM test.using2) USING (a, a, DROP TABLE test.using1; DROP TABLE test.using2; + +-- + +use test; +drop table if exists persons; +drop table if exists children; + +create table persons (id String, name String) engine MergeTree order by id; +create table children (id String, childName String) engine MergeTree order by id; + +insert into persons (id, name) +values ('1', 'John'), ('2', 'Jack'), ('3', 'Daniel'), ('4', 'James'), ('5', 'Amanda'); + +insert into children (id, childName) +values ('1', 'Robert'), ('1', 'Susan'), ('3', 'Sarah'), ('4', 'David'), ('4', 'Joseph'), ('5', 'Robert'); + +select * from persons all inner join children using id; +select * from persons all inner join (select * from children) as j using id; +select * from (select * from persons) as s all inner join (select * from children ) as j using id; +-- +select * from persons all inner join (select * from children) using id; +select * from (select * from persons) all inner join (select * from children) using id; +select * from (select * from persons) as s all inner join (select * from children) using id; + +drop table persons; +drop table children; diff --git a/dbms/tests/queries/0_stateless/00725_join_on_bug_2.sql b/dbms/tests/queries/0_stateless/00725_join_on_bug_2.sql index 116295d967f..7e95aa4a1d3 100644 --- a/dbms/tests/queries/0_stateless/00725_join_on_bug_2.sql +++ b/dbms/tests/queries/0_stateless/00725_join_on_bug_2.sql @@ -9,7 +9,7 @@ insert into test.s values(1,1); select a, b, s_a, s_b from test.t all left join (select a,b,a s_a, b s_b from test.s) using (a,b); select '-'; -select * from test.t all left join test.s using (a,b); +select t.*, s.* from test.t all left join test.s using (a,b); select '-'; select a,b,s_a,s_b from test.t all left join (select a, b, a s_a, b s_b from test.s) s on (s.a = t.a and s.b = t.b); select '-'; diff --git a/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql index e4dabc3a5a6..afaf23b4950 100644 --- a/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql +++ b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql @@ -7,7 +7,7 @@ USE test; CREATE VIEW test AS SELECT 1 AS N; CREATE VIEW test_view AS SELECT * FROM test; CREATE VIEW test_nested_view AS SELECT * FROM (SELECT * FROM test); -CREATE VIEW test_joined_view AS SELECT * FROM test ANY LEFT JOIN test USING N; +CREATE VIEW test_joined_view AS SELECT *, N AS x FROM test ANY LEFT JOIN test USING N; SELECT * FROM test_view; SELECT * FROM test_nested_view; diff --git a/dbms/tests/queries/0_stateless/00800_low_cardinality_join.reference b/dbms/tests/queries/0_stateless/00800_low_cardinality_join.reference index 31d1de2d8c7..8e032c0a542 100644 --- a/dbms/tests/queries/0_stateless/00800_low_cardinality_join.reference +++ b/dbms/tests/queries/0_stateless/00800_low_cardinality_join.reference @@ -1,12 +1,12 @@ -0 0 -0 0 -0 0 -0 0 -0 0 -0 0 -0 0 -0 0 -0 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 - 0 0 0 0 From ec37ef5a89f549ec55d9edc4c90c6bfb97717fd0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:28:13 +0300 Subject: [PATCH 033/324] No escaping for the format schema file's name anymore, added an utility class for parsing and checking the format schema. --- dbms/src/Formats/CapnProtoRowInputStream.cpp | 45 +++++---------- dbms/src/Formats/CapnProtoRowInputStream.h | 3 +- dbms/src/Formats/parseFormatSchema.cpp | 60 ++++++++++++++++++++ dbms/src/Formats/parseFormatSchema.h | 35 ++++++++++++ 4 files changed, 112 insertions(+), 31 deletions(-) create mode 100644 dbms/src/Formats/parseFormatSchema.cpp create mode 100644 dbms/src/Formats/parseFormatSchema.h diff --git a/dbms/src/Formats/CapnProtoRowInputStream.cpp b/dbms/src/Formats/CapnProtoRowInputStream.cpp index 5a2832af9a0..bc3b1bc3926 100644 --- a/dbms/src/Formats/CapnProtoRowInputStream.cpp +++ b/dbms/src/Formats/CapnProtoRowInputStream.cpp @@ -1,12 +1,12 @@ #include #if USE_CAPNP -#include #include #include #include // Y_IGNORE #include #include +#include #include // Y_IGNORE #include // Y_IGNORE #include // Y_IGNORE @@ -26,11 +26,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static String getSchemaPath(const String & schema_dir, const String & schema_file) -{ - return schema_dir + escapeForFileName(schema_file) + ".capnp"; -} - CapnProtoRowInputStream::NestedField split(const Block & header, size_t i) { CapnProtoRowInputStream::NestedField field = {{}, i}; @@ -168,7 +163,7 @@ void CapnProtoRowInputStream::createActions(const NestedFieldList & sorted_field auto node = getFieldOrThrow(cur_reader, field.tokens[parents.size()]); if (node.getType().isList() && actions.size() > 0 && actions.back().field == node) { - // The field list here flattens Nested elements into multiple arrays + // The field list here flattens Nested elements into multiple arrays // In order to map Nested types in Cap'nProto back, they need to be collected // Since the field names are sorted, the order of field positions must be preserved // For example, if the fields are { b @0 :Text, a @1 :Text }, the `a` would come first @@ -184,17 +179,17 @@ void CapnProtoRowInputStream::createActions(const NestedFieldList & sorted_field } } -CapnProtoRowInputStream::CapnProtoRowInputStream(ReadBuffer & istr_, const Block & header_, const String & schema_dir, const String & schema_file, const String & root_object) +CapnProtoRowInputStream::CapnProtoRowInputStream(ReadBuffer & istr_, const Block & header_, const FormatSchemaInfo& info) : istr(istr_), header(header_), parser(std::make_shared()) { // Parse the schema and fetch the root object #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wdeprecated-declarations" - auto schema = parser->impl.parseDiskFile(schema_file, getSchemaPath(schema_dir, schema_file), {}); + auto schema = parser->impl.parseDiskFile(info.schemaPath(), info.absoluteSchemaPath(), {}); #pragma GCC diagnostic pop - root = schema.getNested(root_object).asStruct(); + root = schema.getNested(info.messageName()).asStruct(); /** * The schema typically consists of fields in various nested structures. @@ -298,26 +293,16 @@ bool CapnProtoRowInputStream::read(MutableColumns & columns, RowReadExtension &) void registerInputFormatCapnProto(FormatFactory & factory) { - factory.registerInputFormat("CapnProto", []( - ReadBuffer & buf, - const Block & sample, - const Context & context, - size_t max_block_size, - const FormatSettings & settings) - { - std::vector tokens; - auto schema_and_root = context.getSettingsRef().format_schema.toString(); - boost::split(tokens, schema_and_root, boost::is_any_of(":")); - if (tokens.size() != 2) - throw Exception("Format CapnProto requires 'format_schema' setting to have a schema_file:root_object format, e.g. 'schema.capnp:Message'", - ErrorCodes::BAD_ARGUMENTS); - - const String & schema_dir = context.getFormatSchemaPath(); - - return std::make_shared( - std::make_shared(buf, sample, schema_dir, tokens[0], tokens[1]), - sample, max_block_size, settings); - }); + factory.registerInputFormat( + "CapnProto", + [](ReadBuffer & buf, const Block & sample, const Context & context, size_t max_block_size, const FormatSettings & settings) + { + return std::make_shared( + std::make_shared(buf, sample, FormatSchemaInfo(context, "capnp")), + sample, + max_block_size, + settings); + }); } } diff --git a/dbms/src/Formats/CapnProtoRowInputStream.h b/dbms/src/Formats/CapnProtoRowInputStream.h index a6186d7488d..c40827522aa 100644 --- a/dbms/src/Formats/CapnProtoRowInputStream.h +++ b/dbms/src/Formats/CapnProtoRowInputStream.h @@ -10,6 +10,7 @@ namespace DB { +class FormatSchemaInfo; class ReadBuffer; /** A stream for reading messages in Cap'n Proto format in given schema. @@ -32,7 +33,7 @@ public: * schema_file - location of the capnproto schema, e.g. "schema.capnp" * root_object - name to the root object, e.g. "Message" */ - CapnProtoRowInputStream(ReadBuffer & istr_, const Block & header_, const String & schema_dir, const String & schema_file, const String & root_object); + CapnProtoRowInputStream(ReadBuffer & istr_, const Block & header_, const FormatSchemaInfo & info); bool read(MutableColumns & columns, RowReadExtension &) override; diff --git a/dbms/src/Formats/parseFormatSchema.cpp b/dbms/src/Formats/parseFormatSchema.cpp new file mode 100644 index 00000000000..54c42f8f6ab --- /dev/null +++ b/dbms/src/Formats/parseFormatSchema.cpp @@ -0,0 +1,60 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & schema_file_extension, bool schema_required) +{ + String format_schema = context.getSettingsRef().format_schema.toString(); + if (format_schema.empty()) + { + if (schema_required) + { + throw Exception( + "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + + (schema_file_extension.empty() ? "" : "e.g. 'schema." + schema_file_extension + ":Message'"), + ErrorCodes::BAD_ARGUMENTS); + } + return; + } + + size_t colon_pos = format_schema.find(':'); + if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1)) + { + throw Exception( + "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + + (schema_file_extension.empty() ? "" : "e.g. 'schema." + schema_file_extension + ":Message'") + ". Got '" + format_schema + + "'", + ErrorCodes::BAD_ARGUMENTS); + } + + Poco::Path path(format_schema.substr(0, colon_pos)); + if (context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER)) + { + if (path.isAbsolute()) + throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.toString(), ErrorCodes::BAD_ARGUMENTS); + + if (path.depth() >= 1 && path.directory(0) == "..") + throw Exception( + "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.toString(), + ErrorCodes::BAD_ARGUMENTS); + } + + if (path.getExtension().empty() && !schema_file_extension.empty()) + path.setExtension(schema_file_extension); + + schema_path = path.toString(); + schema_directory = context.getFormatSchemaPath(); + message_name = format_schema.substr(colon_pos + 1); + is_null = false; +} + +} diff --git a/dbms/src/Formats/parseFormatSchema.h b/dbms/src/Formats/parseFormatSchema.h new file mode 100644 index 00000000000..6ad9e6fb2b0 --- /dev/null +++ b/dbms/src/Formats/parseFormatSchema.h @@ -0,0 +1,35 @@ +#pragma once + +#include + +namespace DB +{ +class Context; + +/// Extracts information about where the format schema file is from passed context and keep it. +class FormatSchemaInfo +{ +public: + FormatSchemaInfo() = default; + FormatSchemaInfo(const Context & context, const String & schema_file_extension = String(), bool schema_required = true); + + bool isNull() const { return is_null; } + + /// Returns path to the schema file. + const String & schemaPath() const { return schema_path; } + String absoluteSchemaPath() const { return schema_directory + schema_path; } + + /// Returns directory containing the schema file. + const String & schemaDirectory() const { return schema_directory; } + + /// Returns name of the message type. + const String & messageName() const { return message_name; } + +private: + bool is_null = true; + String schema_path; + String schema_directory; + String message_name; +}; + +} From b7ee466cd77611fd9dba8d2e92afae67a12f5817 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:31:08 +0300 Subject: [PATCH 034/324] Added possibility to set 'format_schema_path' parameter in the client config too. --- dbms/programs/client/Client.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 66acc7fcea5..5da18f3c45d 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -220,6 +220,9 @@ private: APPLY_FOR_SETTINGS(EXTRACT_SETTING) #undef EXTRACT_SETTING + /// Set path for format schema files + if (config().has("format_schema_path")) + context.setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString() + "/"); } From b1aba679d062e87073f3ad88937a749b41c5139f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:32:59 +0300 Subject: [PATCH 035/324] Class WriteBufferFromOwnString now can return StringRef too. --- dbms/src/IO/WriteBufferFromString.h | 3 +++ dbms/src/IO/WriteBufferFromVector.h | 8 ++++++++ 2 files changed, 11 insertions(+) diff --git a/dbms/src/IO/WriteBufferFromString.h b/dbms/src/IO/WriteBufferFromString.h index 0611d284102..c833c7e3f52 100644 --- a/dbms/src/IO/WriteBufferFromString.h +++ b/dbms/src/IO/WriteBufferFromString.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -29,6 +30,8 @@ class WriteBufferFromOwnString : public detail::StringHolder, public WriteBuffer public: WriteBufferFromOwnString() : WriteBufferFromString(value) {} + StringRef stringRef() const { return isFinished() ? StringRef(value) : StringRef(value.data(), pos - value.data()); } + std::string & str() { finish(); diff --git a/dbms/src/IO/WriteBufferFromVector.h b/dbms/src/IO/WriteBufferFromVector.h index 70e6ef5e36e..7cc7eaf25cb 100644 --- a/dbms/src/IO/WriteBufferFromVector.h +++ b/dbms/src/IO/WriteBufferFromVector.h @@ -64,6 +64,14 @@ public: set(nullptr, 0); } + bool isFinished() const { return is_finished; } + + void restart() + { + set(reinterpret_cast(vector.data()), vector.size()); + is_finished = false; + } + ~WriteBufferFromVector() override { if (!is_finished) From 28e983781991559bd28e6c753ef298040e5a05e6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:34:38 +0300 Subject: [PATCH 036/324] Implemented utility class for writing protobufs. --- dbms/src/Formats/ProtobufSimpleWriter.cpp | 412 ++++++++++++++++++++++ dbms/src/Formats/ProtobufSimpleWriter.h | 88 +++++ 2 files changed, 500 insertions(+) create mode 100644 dbms/src/Formats/ProtobufSimpleWriter.cpp create mode 100644 dbms/src/Formats/ProtobufSimpleWriter.h diff --git a/dbms/src/Formats/ProtobufSimpleWriter.cpp b/dbms/src/Formats/ProtobufSimpleWriter.cpp new file mode 100644 index 00000000000..dce58f889f1 --- /dev/null +++ b/dbms/src/Formats/ProtobufSimpleWriter.cpp @@ -0,0 +1,412 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + void writeBytes(WriteBuffer & buf, const void * data, size_t size) { buf.write(reinterpret_cast(data), size); } + + void writeVariant(WriteBuffer & buf, UInt32 value) + { + while (value >= 0x80) + { + buf.write(static_cast(value | 0x80)); + value >>= 7; + } + buf.write(static_cast(value)); + } + + void writeVariant(WriteBuffer & buf, Int32 value) { writeVariant(buf, static_cast(value)); } + + void writeVariant(WriteBuffer & buf, UInt64 value) + { + while (value >= 0x80) + { + buf.write(static_cast(value | 0x80)); + value >>= 7; + } + buf.write(static_cast(value)); + } + + void writeVariant(WriteBuffer & buf, Int64 value) { writeVariant(buf, static_cast(value)); } + + void writeLittleEndian(WriteBuffer & buf, UInt32 value) + { + value = Poco::ByteOrder::toLittleEndian(value); + writeBytes(buf, &value, sizeof(value)); + } + + void writeLittleEndian(WriteBuffer & buf, Int32 value) { writeLittleEndian(buf, static_cast(value)); } + + void writeLittleEndian(WriteBuffer & buf, float value) + { + union + { + Float32 f; + UInt32 i; + }; + f = value; + writeLittleEndian(buf, i); + } + + void writeLittleEndian(WriteBuffer & buf, UInt64 value) + { + value = Poco::ByteOrder::toLittleEndian(value); + writeBytes(buf, &value, sizeof(value)); + } + + void writeLittleEndian(WriteBuffer & buf, Int64 value) { writeLittleEndian(buf, static_cast(value)); } + + void writeLittleEndian(WriteBuffer & buf, double value) + { + union + { + Float64 f; + UInt64 i; + }; + f = value; + writeLittleEndian(buf, i); + } + + UInt32 zigZag(Int32 value) { return (static_cast(value) << 1) ^ static_cast(value >> 31); } + UInt64 zigZag(Int64 value) { return (static_cast(value) << 1) ^ static_cast(value >> 63); } + +} + + +enum ProtobufSimpleWriter::WireType : UInt32 +{ + VARIANT = 0, + BITS64 = 1, + LENGTH_DELIMITED = 2, + BITS32 = 5 +}; + +ProtobufSimpleWriter::ProtobufSimpleWriter(WriteBuffer & out_) : out(out_) +{ +} + +ProtobufSimpleWriter::~ProtobufSimpleWriter() +{ + finishCurrentMessage(); +} + +void ProtobufSimpleWriter::newMessage() +{ + finishCurrentMessage(); + were_messages = true; +} + +void ProtobufSimpleWriter::finishCurrentMessage() +{ + if (!were_messages) + return; + finishCurrentField(); + current_field_number = 0; + StringRef str = message_buffer.stringRef(); + writeVariant(out, str.size); + writeBytes(out, str.data, str.size); + message_buffer.restart(); +} + +void ProtobufSimpleWriter::setCurrentField(UInt32 field_number) +{ + finishCurrentField(); + assert(current_field_number < field_number); + current_field_number = field_number; + num_normal_values = 0; + num_packed_values = 0; +} + +void ProtobufSimpleWriter::finishCurrentField() +{ + if (num_packed_values) + { + assert(!num_normal_values); + StringRef str = repeated_packing_buffer.stringRef(); + if (str.size) + { + writeKey(message_buffer, LENGTH_DELIMITED); + writeVariant(message_buffer, str.size); + writeBytes(message_buffer, str.data, str.size); + repeated_packing_buffer.restart(); + } + } +} + +void ProtobufSimpleWriter::writeKey(WriteBuffer & buf, WireType wire_type) +{ + writeVariant(buf, (current_field_number << 3) | wire_type); +} + +void ProtobufSimpleWriter::writeInt32(Int32 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeUInt32(UInt32 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSInt32(Int32 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, zigZag(value)); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeInt64(Int64 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeUInt64(UInt64 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSInt64(Int64 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, zigZag(value)); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeFixed32(UInt32 value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS32); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSFixed32(Int32 value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS32); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeFloat(float value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS32); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeFixed64(UInt64 value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS64); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSFixed64(Int64 value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS64); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeDouble(double value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS64); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeString(const StringRef & str) +{ + assert(current_field_number); + ++num_normal_values; + writeKey(message_buffer, LENGTH_DELIMITED); + writeVariant(message_buffer, str.size); + writeBytes(message_buffer, str.data, str.size); +} + +void ProtobufSimpleWriter::writeInt32IfNonZero(Int32 value) +{ + if (value) + writeInt32(value); +} + +void ProtobufSimpleWriter::writeUInt32IfNonZero(UInt32 value) +{ + if (value) + writeUInt32(value); +} + +void ProtobufSimpleWriter::writeSInt32IfNonZero(Int32 value) +{ + if (value) + writeSInt32(value); +} + +void ProtobufSimpleWriter::writeInt64IfNonZero(Int64 value) +{ + if (value) + writeInt64(value); +} + +void ProtobufSimpleWriter::writeUInt64IfNonZero(UInt64 value) +{ + if (value) + writeUInt64(value); +} + +void ProtobufSimpleWriter::writeSInt64IfNonZero(Int64 value) +{ + if (value) + writeSInt64(value); +} + +void ProtobufSimpleWriter::writeFixed32IfNonZero(UInt32 value) +{ + if (value) + writeFixed32(value); +} + +void ProtobufSimpleWriter::writeSFixed32IfNonZero(Int32 value) +{ + if (value) + writeSFixed32(value); +} + +void ProtobufSimpleWriter::writeFloatIfNonZero(float value) +{ + if (value != 0) + writeFloat(value); +} + +void ProtobufSimpleWriter::writeFixed64IfNonZero(UInt64 value) +{ + if (value) + writeFixed64(value); +} + +void ProtobufSimpleWriter::writeSFixed64IfNonZero(Int64 value) +{ + if (value) + writeSFixed64(value); +} + +void ProtobufSimpleWriter::writeDoubleIfNonZero(double value) +{ + if (value != 0) + writeDouble(value); +} + +void ProtobufSimpleWriter::writeStringIfNotEmpty(const StringRef & str) +{ + if (str.size) + writeString(str); +} + +void ProtobufSimpleWriter::packRepeatedInt32(Int32 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedUInt32(UInt32 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSInt32(Int32 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, zigZag(value)); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedInt64(Int64 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedUInt64(UInt64 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSInt64(Int64 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, zigZag(value)); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedFixed32(UInt32 value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSFixed32(Int32 value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedFloat(float value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedFixed64(UInt64 value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSFixed64(Int64 value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedDouble(double value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +} diff --git a/dbms/src/Formats/ProtobufSimpleWriter.h b/dbms/src/Formats/ProtobufSimpleWriter.h new file mode 100644 index 00000000000..4880cfb4e21 --- /dev/null +++ b/dbms/src/Formats/ProtobufSimpleWriter.h @@ -0,0 +1,88 @@ +#pragma once + +#include +#include +#include "IO/WriteBufferFromString.h" + + +namespace DB +{ +/** Utility class to serialize protobufs. + * Knows nothing about protobuf schemas, just provides useful functions to serialize data. + * This class is written following the documentation: https://developers.google.com/protocol-buffers/docs/encoding + */ +class ProtobufSimpleWriter : private boost::noncopyable +{ +public: + ProtobufSimpleWriter(WriteBuffer & out_); + ~ProtobufSimpleWriter(); + + /// Should be called when we start writing a new message. + void newMessage(); + + /// Should be called when we start writing a new field. + /// A passed 'field_number' should be positive and greater than any previous 'field_number'. + void setCurrentField(UInt32 field_number); + UInt32 currentFieldNumber() const { return current_field_number; } + + /// Returns number of values added to the current field. + size_t numValues() const { return num_normal_values + num_packed_values; } + + void writeInt32(Int32 value); + void writeUInt32(UInt32 value); + void writeSInt32(Int32 value); + void writeInt64(Int64 value); + void writeUInt64(UInt64 value); + void writeSInt64(Int64 value); + void writeFixed32(UInt32 value); + void writeSFixed32(Int32 value); + void writeFloat(float value); + void writeFixed64(UInt64 value); + void writeSFixed64(Int64 value); + void writeDouble(double value); + void writeString(const StringRef & str); + + void writeInt32IfNonZero(Int32 value); + void writeUInt32IfNonZero(UInt32 value); + void writeSInt32IfNonZero(Int32 value); + void writeInt64IfNonZero(Int64 value); + void writeUInt64IfNonZero(UInt64 value); + void writeSInt64IfNonZero(Int64 value); + void writeFixed32IfNonZero(UInt32 value); + void writeSFixed32IfNonZero(Int32 value); + void writeFloatIfNonZero(float value); + void writeFixed64IfNonZero(UInt64 value); + void writeSFixed64IfNonZero(Int64 value); + void writeDoubleIfNonZero(double value); + void writeStringIfNotEmpty(const StringRef & str); + + void packRepeatedInt32(Int32 value); + void packRepeatedUInt32(UInt32 value); + void packRepeatedSInt32(Int32 value); + void packRepeatedInt64(Int64 value); + void packRepeatedUInt64(UInt64 value); + void packRepeatedSInt64(Int64 value); + void packRepeatedFixed32(UInt32 value); + void packRepeatedSFixed32(Int32 value); + void packRepeatedFloat(float value); + void packRepeatedFixed64(UInt64 value); + void packRepeatedSFixed64(Int64 value); + void packRepeatedDouble(double value); + +private: + void finishCurrentMessage(); + void finishCurrentField(); + + enum WireType : UInt32; + void writeKey(WriteBuffer & buf, WireType wire_type); + + WriteBuffer & out; + bool were_messages = false; + WriteBufferFromOwnString message_buffer; + UInt32 current_field_number = 0; + size_t num_normal_values = 0; + size_t num_packed_values = 0; + WriteBufferFromOwnString repeated_packing_buffer; +}; + +} From 42d9d4e81d482b61e5c9959685ad3bf8c2e1af88 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:36:57 +0300 Subject: [PATCH 037/324] Implemented storage for parsed protobuf schemas. --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Formats/ProtobufSchemas.cpp | 70 ++++++++++++++++++++++++++++ dbms/src/Formats/ProtobufSchemas.h | 49 +++++++++++++++++++ 3 files changed, 120 insertions(+) create mode 100644 dbms/src/Formats/ProtobufSchemas.cpp create mode 100644 dbms/src/Formats/ProtobufSchemas.h diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 13ea9e4744a..1e26a2dd7ed 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -408,6 +408,7 @@ namespace ErrorCodes extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE = 431; extern const int UNKNOWN_CODEC = 432; extern const int ILLEGAL_CODEC_PARAMETER = 433; + extern const int CANNOT_PARSE_PROTOBUF_SCHEMA = 434; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Formats/ProtobufSchemas.cpp b/dbms/src/Formats/ProtobufSchemas.cpp new file mode 100644 index 00000000000..3b60b862319 --- /dev/null +++ b/dbms/src/Formats/ProtobufSchemas.cpp @@ -0,0 +1,70 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; +} + +ProtobufSchemas::ProtobufSchemas() + : disk_source_tree(new google::protobuf::compiler::DiskSourceTree()) + , importer(new google::protobuf::compiler::Importer(disk_source_tree.get(), this)) +{ +} + +ProtobufSchemas::~ProtobufSchemas() = default; + +const google::protobuf::Descriptor * +ProtobufSchemas::getMessageTypeForFormatSchema(const FormatSchemaInfo& info) +{ + // Search the message type among already imported ones. + const auto * descriptor = importer->pool()->FindMessageTypeByName(info.messageName()); + if (descriptor) + return descriptor; + + // Initialize mapping in protobuf's DiskSourceTree. + if (proto_directory.has_value()) + { + assert(*proto_directory == info.schemaDirectory()); // format_schema_path should not be changed! + } + else + { + proto_directory = info.schemaDirectory(); + disk_source_tree->MapPath("", *proto_directory); + } + + const auto * file_descriptor = importer->Import(info.schemaPath()); + + // If there parsing errors AddError() throws an exception and in this case the following line + // isn't executed. + assert(file_descriptor); + + descriptor = file_descriptor->FindMessageTypeByName(info.messageName()); + if (!descriptor) + throw Exception( + "Not found a message named '" + info.messageName() + "' in the schema file '" + info.schemaPath() + "'", + ErrorCodes::BAD_ARGUMENTS); + return descriptor; +} + +const google::protobuf::Descriptor * ProtobufSchemas::getMessageTypeForColumns(const std::vector & /*columns*/) +{ + throw Exception("Using the 'Protobuf' format without schema is not implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +void ProtobufSchemas::AddError(const String & filename, int line, int column, const String & message) +{ + throw Exception( + "Cannot parse '" + filename + "' file, found an error at line " + std::to_string(line) + ", column " + std::to_string(column) + ", " + + message, + ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA); +} + +} diff --git a/dbms/src/Formats/ProtobufSchemas.h b/dbms/src/Formats/ProtobufSchemas.h new file mode 100644 index 00000000000..b67f03f5270 --- /dev/null +++ b/dbms/src/Formats/ProtobufSchemas.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include +#include + + +namespace google +{ +namespace protobuf +{ + class Descriptor; +} +} + +namespace DB +{ +class Block; +class FormatSchemaInfo; +struct ColumnWithTypeAndName; + +/** Keeps parsed google protobuf schemas either parsed from files or generated from DB columns. + * This class is used to handle the "Protobuf" input/output formats. + */ +class ProtobufSchemas : public ext::singleton, public google::protobuf::compiler::MultiFileErrorCollector +{ +public: + ProtobufSchemas(); + ~ProtobufSchemas() override; + + /// Parses the format schema, then parses the corresponding proto file, and returns the descriptor of the message type. + /// The function never returns nullptr, it throws an exception if it cannot load or parse the file. + const google::protobuf::Descriptor * getMessageTypeForFormatSchema(const FormatSchemaInfo& info); + + /// Generates a message type with suitable types of fields to store a block with |header|, then returns the descriptor + /// of the generated message type. + const google::protobuf::Descriptor * getMessageTypeForColumns(const std::vector & columns); + +private: + // Overrides google::protobuf::compiler::MultiFileErrorCollector: + void AddError(const String & filename, int line, int column, const String & message) override; + + std::optional proto_directory; + std::unique_ptr disk_source_tree; + std::unique_ptr importer; +}; + +} From 1e7a231c56d7fc1e4a3ea84eacda7755f988c954 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:39:19 +0300 Subject: [PATCH 038/324] Implemented main class for writing protobufs. --- dbms/src/Common/ErrorCodes.cpp | 3 + dbms/src/Formats/ProtobufWriter.cpp | 809 ++++++++++++++++++++++++++++ dbms/src/Formats/ProtobufWriter.h | 102 ++++ 3 files changed, 914 insertions(+) create mode 100644 dbms/src/Formats/ProtobufWriter.cpp create mode 100644 dbms/src/Formats/ProtobufWriter.h diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 1e26a2dd7ed..f06a88c96da 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -409,6 +409,9 @@ namespace ErrorCodes extern const int UNKNOWN_CODEC = 432; extern const int ILLEGAL_CODEC_PARAMETER = 433; extern const int CANNOT_PARSE_PROTOBUF_SCHEMA = 434; + extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD = 435; + extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE = 436; + extern const int PROTOBUF_FIELD_NOT_REPEATED = 437; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Formats/ProtobufWriter.cpp b/dbms/src/Formats/ProtobufWriter.cpp new file mode 100644 index 00000000000..d17754e3478 --- /dev/null +++ b/dbms/src/Formats/ProtobufWriter.cpp @@ -0,0 +1,809 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD; + extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE; + extern const int PROTOBUF_FIELD_NOT_REPEATED; +} + + +// Converter ----------------------------------------------------------------------------------------------------------- + +class ProtobufWriter::Converter : private boost::noncopyable +{ +public: + Converter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : simple_writer(simple_writer_), field(field_) + { + } + + virtual ~Converter() = default; + + virtual void writeString(const StringRef &) { cannotConvertType("String"); } + + virtual void writeInt8(Int8) { cannotConvertType("Int8"); } + virtual void writeUInt8(UInt8) { cannotConvertType("UInt8"); } + virtual void writeInt16(Int16) { cannotConvertType("Int16"); } + virtual void writeUInt16(UInt16) { cannotConvertType("UInt16"); } + virtual void writeInt32(Int32) { cannotConvertType("Int32"); } + virtual void writeUInt32(UInt32) { cannotConvertType("UInt32"); } + virtual void writeInt64(Int64) { cannotConvertType("Int64"); } + virtual void writeUInt64(UInt64) { cannotConvertType("UInt64"); } + virtual void writeUInt128(const UInt128 &) { cannotConvertType("UInt128"); } + virtual void writeFloat32(Float32) { cannotConvertType("Float32"); } + virtual void writeFloat64(Float64) { cannotConvertType("Float64"); } + + virtual void prepareEnumMappingInt8(const std::vector> &) {} + virtual void prepareEnumMappingInt16(const std::vector> &) {} + virtual void writeEnumInt8(Int8) { cannotConvertType("Enum"); } + virtual void writeEnumInt16(Int16) { cannotConvertType("Enum"); } + + virtual void writeUUID(const UUID &) { cannotConvertType("UUID"); } + virtual void writeDate(DayNum) { cannotConvertType("Date"); } + virtual void writeDateTime(time_t) { cannotConvertType("DateTime"); } + + virtual void writeDecimal32(Decimal32, UInt32) { cannotConvertType("Decimal32"); } + virtual void writeDecimal64(Decimal64, UInt32) { cannotConvertType("Decimal64"); } + virtual void writeDecimal128(const Decimal128 &, UInt32) { cannotConvertType("Decimal128"); } + + virtual void writeAggregateFunction(const AggregateFunctionPtr &, ConstAggregateDataPtr) { cannotConvertType("AggregateFunction"); } + +protected: + void cannotConvertType(const String & type_name) + { + throw Exception( + "Could not convert data type '" + type_name + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")", + ErrorCodes::CANNOT_CONVERT_TO_PROTOBUF_TYPE); + } + + void cannotConvertValue(const String & value) + { + throw Exception( + "Could not convert value '" + value + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")", + ErrorCodes::CANNOT_CONVERT_TO_PROTOBUF_TYPE); + } + + template + To numericCast(From value) + { + if constexpr (std::is_same_v) + return value; + To result; + try + { + result = boost::numeric_cast(value); + } + catch (boost::numeric::bad_numeric_cast &) + { + cannotConvertValue(toString(value)); + } + return result; + } + + template + To parseFromString(const StringRef & str) + { + To result; + try + { + result = ::DB::parse(str.data, str.size); + } + catch (...) + { + cannotConvertValue(str.toString()); + } + return result; + } + + bool packRepeated() const + { + if (!field->is_repeated()) + return false; + if (field->options().has_packed()) + return field->options().packed(); + return field->file()->syntax() == google::protobuf::FileDescriptor::SYNTAX_PROTO3; + } + + bool skipNullValue() const + { + return field->is_optional() && (field->file()->syntax() == google::protobuf::FileDescriptor::SYNTAX_PROTO3); + } + + ProtobufSimpleWriter & simple_writer; + const google::protobuf::FieldDescriptor * field; +}; + + +// ToStringConverter --------------------------------------------------------------------------------------------------- + +class ProtobufWriter::ToStringConverter : public Converter +{ +public: + ToStringConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override { writeField(str); } + + void writeInt8(Int8 value) override { convertToStringAndWriteField(value); } + void writeUInt8(UInt8 value) override { convertToStringAndWriteField(value); } + void writeInt16(Int16 value) override { convertToStringAndWriteField(value); } + void writeUInt16(UInt16 value) override { convertToStringAndWriteField(value); } + void writeInt32(Int32 value) override { convertToStringAndWriteField(value); } + void writeUInt32(UInt32 value) override { convertToStringAndWriteField(value); } + void writeInt64(Int64 value) override { convertToStringAndWriteField(value); } + void writeUInt64(UInt64 value) override { convertToStringAndWriteField(value); } + void writeFloat32(Float32 value) override { convertToStringAndWriteField(value); } + void writeFloat64(Float64 value) override { convertToStringAndWriteField(value); } + + void prepareEnumMappingInt8(const std::vector> & name_value_pairs) override + { + prepareEnumValueToNameMap(name_value_pairs); + } + void prepareEnumMappingInt16(const std::vector> & name_value_pairs) override + { + prepareEnumValueToNameMap(name_value_pairs); + } + + void writeEnumInt8(Int8 value) override { writeEnumInt16(value); } + + void writeEnumInt16(Int16 value) override + { + auto it = enum_value_to_name_map->find(value); + if (it == enum_value_to_name_map->end()) + cannotConvertValue(toString(value)); + writeField(it->second); + } + + void writeUUID(const UUID & uuid) override { convertToStringAndWriteField(uuid); } + void writeDate(DayNum date) override { convertToStringAndWriteField(date); } + + void writeDateTime(time_t tm) override + { + writeDateTimeText(tm, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + + void writeDecimal32(Decimal32 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + void writeDecimal64(Decimal64 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + void writeDecimal128(const Decimal128 & decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + + void writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) override + { + function->serialize(place, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + +private: + template + void convertToStringAndWriteField(T value) + { + writeText(value, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + + template + void writeDecimal(const Decimal & decimal, UInt32 scale) + { + writeText(decimal, scale, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + + template + void prepareEnumValueToNameMap(const std::vector> & name_value_pairs) + { + if (enum_value_to_name_map.has_value()) + return; + enum_value_to_name_map.emplace(); + for (const auto & name_value_pair : name_value_pairs) + enum_value_to_name_map->emplace(name_value_pair.second, name_value_pair.first); + } + + void writeField(const StringRef & str) { (simple_writer.*write_field_function)(str); } + + void initWriteFieldFunction() + { + write_field_function = skipNullValue() ? &ProtobufSimpleWriter::writeStringIfNotEmpty : &ProtobufSimpleWriter::writeString; + } + + void (ProtobufSimpleWriter::*write_field_function)(const StringRef & str); + WriteBufferFromOwnString text_buffer; + std::optional> enum_value_to_name_map; +}; + + +// ToNumberConverter --------------------------------------------------------------------------------------------------- + +template +class ProtobufWriter::ToNumberConverter : public Converter +{ +public: + ToNumberConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override { writeField(parseFromString(str)); } + + void writeInt8(Int8 value) override { castNumericAndWriteField(value); } + void writeUInt8(UInt8 value) override { castNumericAndWriteField(value); } + void writeInt16(Int16 value) override { castNumericAndWriteField(value); } + void writeUInt16(UInt16 value) override { castNumericAndWriteField(value); } + void writeInt32(Int32 value) override { castNumericAndWriteField(value); } + void writeUInt32(UInt32 value) override { castNumericAndWriteField(value); } + void writeInt64(Int64 value) override { castNumericAndWriteField(value); } + void writeUInt64(UInt64 value) override { castNumericAndWriteField(value); } + void writeFloat32(Float32 value) override { castNumericAndWriteField(value); } + void writeFloat64(Float64 value) override { castNumericAndWriteField(value); } + + void writeEnumInt8(Int8 value) override { writeEnumInt16(value); } + + void writeEnumInt16(Int16 value) override + { + if constexpr (!std::is_integral_v) + cannotConvertType("Enum"); // It's not correct to convert enum to floating point. + castNumericAndWriteField(value); + } + + void writeDate(DayNum date) override { castNumericAndWriteField(static_cast(date)); } + void writeDateTime(time_t tm) override { castNumericAndWriteField(tm); } + + void writeDecimal32(Decimal32 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + void writeDecimal64(Decimal64 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + +private: + template + void castNumericAndWriteField(From value) + { + writeField(numericCast(value)); + } + + template + void writeDecimal(const Decimal & decimal, UInt32 scale) + { + if constexpr (std::is_integral_v) + castNumericAndWriteField(decimal.value / decimalScaleMultiplier(scale)); + else + castNumericAndWriteField(double(decimal.value) * pow(10., -double(scale))); + } + + void writeField(T value) { (simple_writer.*write_field_function)(value); } + + void initWriteFieldFunction() + { + if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_INT32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeInt32IfNonZero : &ProtobufSimpleWriter::writeInt32); + break; + case google::protobuf::FieldDescriptor::TYPE_SINT32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSInt32IfNonZero : &ProtobufSimpleWriter::writeSInt32); + break; + case google::protobuf::FieldDescriptor::TYPE_SFIXED32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSFixed32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSFixed32IfNonZero : &ProtobufSimpleWriter::writeSFixed32); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_UINT32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt32IfNonZero : &ProtobufSimpleWriter::writeUInt32); + break; + case google::protobuf::FieldDescriptor::TYPE_FIXED32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedFixed32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeFixed32IfNonZero : &ProtobufSimpleWriter::writeFixed32); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_INT64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedInt64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeInt64IfNonZero : &ProtobufSimpleWriter::writeInt64); + break; + case google::protobuf::FieldDescriptor::TYPE_SINT64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSInt64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSInt64IfNonZero : &ProtobufSimpleWriter::writeSInt64); + break; + case google::protobuf::FieldDescriptor::TYPE_SFIXED64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSFixed64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSFixed64IfNonZero : &ProtobufSimpleWriter::writeSFixed64); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_UINT64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt64IfNonZero : &ProtobufSimpleWriter::writeUInt64); + break; + case google::protobuf::FieldDescriptor::TYPE_FIXED64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedFixed64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeFixed64IfNonZero : &ProtobufSimpleWriter::writeFixed64); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedFloat + : (skipNullValue() ? &ProtobufSimpleWriter::writeFloatIfNonZero : &ProtobufSimpleWriter::writeFloat); + } + else if constexpr (std::is_same_v) + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedDouble + : (skipNullValue() ? &ProtobufSimpleWriter::writeDoubleIfNonZero : &ProtobufSimpleWriter::writeDouble); + } + else + { + assert(false); + } + } + + void (ProtobufSimpleWriter::*write_field_function)(T value); +}; + + +// ToBoolConverter ----------------------------------------------------------------------------------------------------- + +class ProtobufWriter::ToBoolConverter : public Converter +{ +public: + ToBoolConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override + { + if (str == "true") + writeField(true); + else if (str == "false") + writeField(false); + else + cannotConvertValue(str.toString()); + } + + void writeInt8(Int8 value) override { convertToBoolAndWriteField(value); } + void writeUInt8(UInt8 value) override { convertToBoolAndWriteField(value); } + void writeInt16(Int16 value) override { convertToBoolAndWriteField(value); } + void writeUInt16(UInt16 value) override { convertToBoolAndWriteField(value); } + void writeInt32(Int32 value) override { convertToBoolAndWriteField(value); } + void writeUInt32(UInt32 value) override { convertToBoolAndWriteField(value); } + void writeInt64(Int64 value) override { convertToBoolAndWriteField(value); } + void writeUInt64(UInt64 value) override { convertToBoolAndWriteField(value); } + void writeFloat32(Float32 value) override { convertToBoolAndWriteField(value); } + void writeFloat64(Float64 value) override { convertToBoolAndWriteField(value); } + void writeDecimal32(Decimal32 decimal, UInt32) override { convertToBoolAndWriteField(decimal.value); } + void writeDecimal64(Decimal64 decimal, UInt32) override { convertToBoolAndWriteField(decimal.value); } + void writeDecimal128(const Decimal128 & decimal, UInt32) override { convertToBoolAndWriteField(decimal.value); } + +private: + template + void convertToBoolAndWriteField(T value) + { + writeField(static_cast(value)); + } + + void writeField(bool b) { (simple_writer.*write_field_function)(b); } + + void initWriteFieldFunction() + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt32IfNonZero : &ProtobufSimpleWriter::writeUInt32); + } + + void (ProtobufSimpleWriter::*write_field_function)(UInt32 b); +}; + + +// ToEnumConverter ----------------------------------------------------------------------------------------------------- + +class ProtobufWriter::ToEnumConverter : public Converter +{ +public: + ToEnumConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override + { + prepareEnumNameToPbNumberMap(); + auto it = enum_name_to_pbnumber_map->find(str); + if (it == enum_name_to_pbnumber_map->end()) + cannotConvertValue(str.toString()); + writeField(it->second); + } + + void writeInt8(Int8 value) override { convertToEnumAndWriteField(value); } + void writeUInt8(UInt8 value) override { convertToEnumAndWriteField(value); } + void writeInt16(Int16 value) override { convertToEnumAndWriteField(value); } + void writeUInt16(UInt16 value) override { convertToEnumAndWriteField(value); } + void writeInt32(Int32 value) override { convertToEnumAndWriteField(value); } + void writeUInt32(UInt32 value) override { convertToEnumAndWriteField(value); } + void writeInt64(Int64 value) override { convertToEnumAndWriteField(value); } + void writeUInt64(UInt64 value) override { convertToEnumAndWriteField(value); } + + void prepareEnumMappingInt8(const std::vector> & name_value_pairs) override + { + prepareEnumValueToPbNumberMap(name_value_pairs); + } + void prepareEnumMappingInt16(const std::vector> & name_value_pairs) override + { + prepareEnumValueToPbNumberMap(name_value_pairs); + } + + void writeEnumInt8(Int8 value) override { writeEnumInt16(value); } + + void writeEnumInt16(Int16 value) override + { + auto it = enum_value_to_pbnumber_map->find(value); + if (it == enum_value_to_pbnumber_map->end()) + cannotConvertValue(toString(value)); + writeField(it->second); + } + +private: + template + void convertToEnumAndWriteField(T value) + { + const auto * enum_descriptor = field->enum_type()->FindValueByNumber(numericCast(value)); + if (!enum_descriptor) + cannotConvertValue(toString(value)); + writeField(enum_descriptor->number()); + } + + void prepareEnumNameToPbNumberMap() + { + if (enum_name_to_pbnumber_map.has_value()) + return; + enum_name_to_pbnumber_map.emplace(); + const auto * enum_type = field->enum_type(); + for (int i = 0; i != enum_type->value_count(); ++i) + { + const auto * enum_value = enum_type->value(i); + enum_name_to_pbnumber_map->emplace(enum_value->name(), enum_value->number()); + } + } + + template + void prepareEnumValueToPbNumberMap(const std::vector> & name_value_pairs) + { + if (enum_value_to_pbnumber_map.has_value()) + return; + enum_value_to_pbnumber_map.emplace(); + for (const auto & name_value_pair : name_value_pairs) + { + Int16 value = name_value_pair.second; + const auto * enum_descriptor = field->enum_type()->FindValueByName(name_value_pair.first); + if (enum_descriptor) + enum_value_to_pbnumber_map->emplace(value, enum_descriptor->number()); + } + } + + void writeField(int enum_number) { (simple_writer.*write_field_function)(enum_number); } + + void initWriteFieldFunction() + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt32IfNonZero : &ProtobufSimpleWriter::writeUInt32); + } + + void (ProtobufSimpleWriter::*write_field_function)(UInt32 enum_number); + std::optional> enum_name_to_pbnumber_map; + std::optional> enum_value_to_pbnumber_map; +}; + + +// ProtobufWriter ------------------------------------------------------------------------------------------------------ + +ProtobufWriter::ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type) : simple_writer(out) +{ + enumerateFieldsInWriteOrder(message_type); + createConverters(); +} + +ProtobufWriter::~ProtobufWriter() +{ + finishCurrentMessage(); +} + +void ProtobufWriter::enumerateFieldsInWriteOrder(const google::protobuf::Descriptor * message_type) +{ + assert(fields_in_write_order.empty()); + fields_in_write_order.reserve(message_type->field_count()); + for (int i = 0; i < message_type->field_count(); ++i) + fields_in_write_order.emplace_back(message_type->field(i)); + + std::sort( + fields_in_write_order.begin(), + fields_in_write_order.end(), + [](const google::protobuf::FieldDescriptor * left, const google::protobuf::FieldDescriptor * right) + { + return left->number() < right->number(); + }); +} + +void ProtobufWriter::createConverters() +{ + assert(converters.empty()); + converters.reserve(fields_in_write_order.size()); + for (size_t i = 0; i != fields_in_write_order.size(); ++i) + { + const auto * field = fields_in_write_order[i]; + std::unique_ptr converter; + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_INT32: + case google::protobuf::FieldDescriptor::TYPE_SINT32: + case google::protobuf::FieldDescriptor::TYPE_SFIXED32: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_UINT32: + case google::protobuf::FieldDescriptor::TYPE_FIXED32: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_INT64: + case google::protobuf::FieldDescriptor::TYPE_SFIXED64: + case google::protobuf::FieldDescriptor::TYPE_SINT64: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_UINT64: + case google::protobuf::FieldDescriptor::TYPE_FIXED64: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_FLOAT: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_DOUBLE: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_BOOL: + converter = std::make_unique(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_ENUM: + converter = std::make_unique(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_STRING: + case google::protobuf::FieldDescriptor::TYPE_BYTES: + converter = std::make_unique(simple_writer, field); + break; + default: + throw Exception(String("Protobuf type '") + field->type_name() + "' isn't supported", ErrorCodes::NOT_IMPLEMENTED); + } + converters.emplace_back(std::move(converter)); + } +} + +const std::vector & ProtobufWriter::fieldsInWriteOrder() const +{ + return fields_in_write_order; +} + +void ProtobufWriter::newMessage() +{ + finishCurrentMessage(); + simple_writer.newMessage(); + if (fields_in_write_order.empty()) + return; + + current_field_index = 0; + current_field = fields_in_write_order[current_field_index]; + current_converter = converters[current_field_index].get(); + simple_writer.setCurrentField(current_field->number()); +} + +void ProtobufWriter::finishCurrentMessage() +{ + if (current_field) + { + assert(current_field_index == fields_in_write_order.size() - 1); + finishCurrentField(); + } +} + +bool ProtobufWriter::nextField() +{ + if (current_field_index == fields_in_write_order.size() - 1) + return false; + + finishCurrentField(); + + ++current_field_index; + current_field = fields_in_write_order[current_field_index]; + current_converter = converters[current_field_index].get(); + simple_writer.setCurrentField(current_field->number()); + return true; +} + +void ProtobufWriter::finishCurrentField() +{ + assert(current_field); + size_t num_values = simple_writer.numValues(); + if (num_values == 0) + { + if (current_field->is_required()) + throw Exception( + "No data for the required field '" + current_field->name() + "'", ErrorCodes::NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD); + } + else if (num_values > 1 && !current_field->is_repeated()) + { + throw Exception( + "Cannot write more than single value to the non-repeated field '" + current_field->name() + "'", + ErrorCodes::PROTOBUF_FIELD_NOT_REPEATED); + } +} + +void ProtobufWriter::writeNumber(Int8 value) +{ + current_converter->writeInt8(value); +} + +void ProtobufWriter::writeNumber(UInt8 value) +{ + current_converter->writeUInt8(value); +} + +void ProtobufWriter::writeNumber(Int16 value) +{ + current_converter->writeInt16(value); +} + +void ProtobufWriter::writeNumber(UInt16 value) +{ + current_converter->writeUInt16(value); +} + +void ProtobufWriter::writeNumber(Int32 value) +{ + current_converter->writeInt32(value); +} + +void ProtobufWriter::writeNumber(UInt32 value) +{ + current_converter->writeUInt32(value); +} + +void ProtobufWriter::writeNumber(Int64 value) +{ + current_converter->writeInt64(value); +} + +void ProtobufWriter::writeNumber(UInt64 value) +{ + current_converter->writeUInt64(value); +} + +void ProtobufWriter::writeNumber(UInt128 value) +{ + current_converter->writeUInt128(value); +} + +void ProtobufWriter::writeNumber(Float32 value) +{ + current_converter->writeFloat32(value); +} + +void ProtobufWriter::writeNumber(Float64 value) +{ + current_converter->writeFloat64(value); +} + +void ProtobufWriter::writeString(const StringRef & str) +{ + current_converter->writeString(str); +} + +void ProtobufWriter::prepareEnumMapping(const std::vector> & enum_values) +{ + current_converter->prepareEnumMappingInt8(enum_values); +} + +void ProtobufWriter::prepareEnumMapping(const std::vector> & enum_values) +{ + current_converter->prepareEnumMappingInt16(enum_values); +} + +void ProtobufWriter::writeEnum(Int8 value) +{ + current_converter->writeEnumInt8(value); +} + +void ProtobufWriter::writeEnum(Int16 value) +{ + current_converter->writeEnumInt16(value); +} + +void ProtobufWriter::writeUUID(const UUID & uuid) +{ + current_converter->writeUUID(uuid); +} + +void ProtobufWriter::writeDate(DayNum date) +{ + current_converter->writeDate(date); +} + +void ProtobufWriter::writeDateTime(time_t tm) +{ + current_converter->writeDateTime(tm); +} + +void ProtobufWriter::writeDecimal(Decimal32 decimal, UInt32 scale) +{ + current_converter->writeDecimal32(decimal, scale); +} + +void ProtobufWriter::writeDecimal(Decimal64 decimal, UInt32 scale) +{ + current_converter->writeDecimal64(decimal, scale); +} + +void ProtobufWriter::writeDecimal(const Decimal128 & decimal, UInt32 scale) +{ + current_converter->writeDecimal128(decimal, scale); +} + +void ProtobufWriter::writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) +{ + current_converter->writeAggregateFunction(function, place); +} + +} diff --git a/dbms/src/Formats/ProtobufWriter.h b/dbms/src/Formats/ProtobufWriter.h new file mode 100644 index 00000000000..a6be952dc2b --- /dev/null +++ b/dbms/src/Formats/ProtobufWriter.h @@ -0,0 +1,102 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace google +{ +namespace protobuf +{ + class Descriptor; + class FieldDescriptor; +} +} + + +namespace DB +{ +class IAggregateFunction; +using AggregateFunctionPtr = std::shared_ptr; +using ConstAggregateDataPtr = const char *; + + +/** Serializes a protobuf, tries to cast types if necessarily. + */ +class ProtobufWriter : private boost::noncopyable +{ +public: + ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type); + ~ProtobufWriter(); + + /// Returns fields of the protobuf schema sorted by their numbers. + const std::vector & fieldsInWriteOrder() const; + + /// Should be called when we start writing a new message. + void newMessage(); + + /// Should be called when we start writing a new field. + /// Returns false if there is no more fields in the message type. + bool nextField(); + + /// Returns the current field of the message type. + /// The value returned by this function changes after calling nextField() or newMessage(). + const google::protobuf::FieldDescriptor * currentField() const { return current_field; } + + void writeNumber(Int8 value); + void writeNumber(UInt8 value); + void writeNumber(Int16 value); + void writeNumber(UInt16 value); + void writeNumber(Int32 value); + void writeNumber(UInt32 value); + void writeNumber(Int64 value); + void writeNumber(UInt64 value); + void writeNumber(UInt128 value); + void writeNumber(Float32 value); + void writeNumber(Float64 value); + + void writeString(const StringRef & value); + + void prepareEnumMapping(const std::vector> & name_value_pairs); + void prepareEnumMapping(const std::vector> & name_value_pairs); + void writeEnum(Int8 value); + void writeEnum(Int16 value); + + void writeUUID(const UUID & value); + void writeDate(DayNum date); + void writeDateTime(time_t tm); + + void writeDecimal(Decimal32 decimal, UInt32 scale); + void writeDecimal(Decimal64 decimal, UInt32 scale); + void writeDecimal(const Decimal128 & decimal, UInt32 scale); + + void writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place); + +private: + void enumerateFieldsInWriteOrder(const google::protobuf::Descriptor * message_type); + void createConverters(); + + void finishCurrentMessage(); + void finishCurrentField(); + + ProtobufSimpleWriter simple_writer; + std::vector fields_in_write_order; + size_t current_field_index = -1; + const google::protobuf::FieldDescriptor * current_field = nullptr; + + class Converter; + class ToStringConverter; + template + class ToNumberConverter; + class ToBoolConverter; + class ToEnumConverter; + + std::vector> converters; + Converter * current_converter = nullptr; +}; + +} From c241a4ea6d31167dc7b4c7b0197b40d969fd2ea5 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:41:18 +0300 Subject: [PATCH 039/324] Added function serializeProtobuf() to each data type. --- .../DataTypes/DataTypeAggregateFunction.cpp | 7 +++ .../src/DataTypes/DataTypeAggregateFunction.h | 1 + dbms/src/DataTypes/DataTypeArray.cpp | 12 +++++ dbms/src/DataTypes/DataTypeArray.h | 4 ++ dbms/src/DataTypes/DataTypeDate.cpp | 6 +++ dbms/src/DataTypes/DataTypeDate.h | 1 + dbms/src/DataTypes/DataTypeDateTime.cpp | 6 +++ dbms/src/DataTypes/DataTypeDateTime.h | 1 + dbms/src/DataTypes/DataTypeEnum.cpp | 8 ++++ dbms/src/DataTypes/DataTypeEnum.h | 2 + dbms/src/DataTypes/DataTypeFixedString.cpp | 8 ++++ dbms/src/DataTypes/DataTypeFixedString.h | 2 + dbms/src/DataTypes/DataTypeLowCardinality.cpp | 12 +++-- dbms/src/DataTypes/DataTypeLowCardinality.h | 47 ++++++++++--------- dbms/src/DataTypes/DataTypeNullable.cpp | 7 +++ dbms/src/DataTypes/DataTypeNullable.h | 2 + dbms/src/DataTypes/DataTypeNumberBase.cpp | 11 ++++- dbms/src/DataTypes/DataTypeNumberBase.h | 3 +- dbms/src/DataTypes/DataTypeString.cpp | 8 +++- dbms/src/DataTypes/DataTypeString.h | 2 + dbms/src/DataTypes/DataTypeTuple.cpp | 6 +++ dbms/src/DataTypes/DataTypeTuple.h | 2 + dbms/src/DataTypes/DataTypeUUID.cpp | 6 +++ dbms/src/DataTypes/DataTypeUUID.h | 1 + dbms/src/DataTypes/DataTypesDecimal.cpp | 8 ++++ dbms/src/DataTypes/DataTypesDecimal.h | 2 + dbms/src/DataTypes/IDataType.h | 5 ++ dbms/src/DataTypes/IDataTypeDummy.h | 1 + ...eFormatSchema.cpp => FormatSchemaInfo.cpp} | 0 ...parseFormatSchema.h => FormatSchemaInfo.h} | 0 30 files changed, 152 insertions(+), 29 deletions(-) rename dbms/src/Formats/{parseFormatSchema.cpp => FormatSchemaInfo.cpp} (100%) rename dbms/src/Formats/{parseFormatSchema.h => FormatSchemaInfo.h} (100%) diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp index 7a1b163f3b6..19800db3b05 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -248,6 +249,12 @@ void DataTypeAggregateFunction::deserializeTextCSV(IColumn & column, ReadBuffer } +void DataTypeAggregateFunction::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeAggregateFunction(function, static_cast(column).getData()[row_num]); +} + + MutableColumnPtr DataTypeAggregateFunction::createColumn() const { return ColumnAggregateFunction::create(function); diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.h b/dbms/src/DataTypes/DataTypeAggregateFunction.h index d68f460dea6..a21f5d65125 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.h +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.h @@ -56,6 +56,7 @@ public: void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; MutableColumnPtr createColumn() const override; diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 0b985039ccf..a1677d9bdbd 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -430,6 +430,18 @@ void DataTypeArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons } +void DataTypeArray::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + const ColumnArray & column_array = static_cast(column); + const ColumnArray::Offsets & offsets = column_array.getOffsets(); + size_t offset = offsets[row_num - 1]; + size_t next_offset = offsets[row_num]; + const IColumn & nested_column = column_array.getData(); + for (size_t i = offset; i < next_offset; ++i) + nested->serializeProtobuf(nested_column, i, protobuf); +} + + MutableColumnPtr DataTypeArray::createColumn() const { return ColumnArray::create(nested->createColumn(), ColumnArray::ColumnOffsets::create()); diff --git a/dbms/src/DataTypes/DataTypeArray.h b/dbms/src/DataTypes/DataTypeArray.h index 5a89a5d8e5a..4c04a2624f5 100644 --- a/dbms/src/DataTypes/DataTypeArray.h +++ b/dbms/src/DataTypes/DataTypeArray.h @@ -84,6 +84,10 @@ public: DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; + void serializeProtobuf(const IColumn & column, + size_t row_num, + ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override; diff --git a/dbms/src/DataTypes/DataTypeDate.cpp b/dbms/src/DataTypes/DataTypeDate.cpp index 688e3c91db3..d5556cb3378 100644 --- a/dbms/src/DataTypes/DataTypeDate.cpp +++ b/dbms/src/DataTypes/DataTypeDate.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -72,6 +73,11 @@ void DataTypeDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const static_cast(column).getData().push_back(value.getDayNum()); } +void DataTypeDate::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeDate(DayNum(static_cast(column).getData()[row_num])); +} + bool DataTypeDate::equals(const IDataType & rhs) const { return typeid(rhs) == typeid(*this); diff --git a/dbms/src/DataTypes/DataTypeDate.h b/dbms/src/DataTypes/DataTypeDate.h index 89d773149c5..9bc56cc3762 100644 --- a/dbms/src/DataTypes/DataTypeDate.h +++ b/dbms/src/DataTypes/DataTypeDate.h @@ -21,6 +21,7 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; bool canBeUsedAsVersion() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/dbms/src/DataTypes/DataTypeDateTime.cpp b/dbms/src/DataTypes/DataTypeDateTime.cpp index 6ea042454e5..e894eb79252 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.cpp +++ b/dbms/src/DataTypes/DataTypeDateTime.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -138,6 +139,11 @@ void DataTypeDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c static_cast(column).getData().push_back(x); } +void DataTypeDateTime::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeDateTime(static_cast(column).getData()[row_num]); +} + bool DataTypeDateTime::equals(const IDataType & rhs) const { /// DateTime with different timezones are equal, because: diff --git a/dbms/src/DataTypes/DataTypeDateTime.h b/dbms/src/DataTypes/DataTypeDateTime.h index 95b120c6c9d..10fe80a8213 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.h +++ b/dbms/src/DataTypes/DataTypeDateTime.h @@ -46,6 +46,7 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; bool canBeUsedAsVersion() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/dbms/src/DataTypes/DataTypeEnum.cpp b/dbms/src/DataTypes/DataTypeEnum.cpp index 3cd6af46241..f6566ad9040 100644 --- a/dbms/src/DataTypes/DataTypeEnum.cpp +++ b/dbms/src/DataTypes/DataTypeEnum.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -222,6 +223,13 @@ void DataTypeEnum::deserializeBinaryBulk( x.resize(initial_size + size / sizeof(FieldType)); } +template +void DataTypeEnum::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.prepareEnumMapping(values); + protobuf.writeEnum(static_cast(column).getData()[row_num]); +} + template Field DataTypeEnum::getDefault() const { diff --git a/dbms/src/DataTypes/DataTypeEnum.h b/dbms/src/DataTypes/DataTypeEnum.h index 43e4c578f76..f25d387c818 100644 --- a/dbms/src/DataTypes/DataTypeEnum.h +++ b/dbms/src/DataTypes/DataTypeEnum.h @@ -105,6 +105,8 @@ public: void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, const size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, const size_t limit, const double avg_value_size_hint) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override { return ColumnType::create(); } Field getDefault() const override; diff --git a/dbms/src/DataTypes/DataTypeFixedString.cpp b/dbms/src/DataTypes/DataTypeFixedString.cpp index f43de616e8d..6fbea93efbd 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.cpp +++ b/dbms/src/DataTypes/DataTypeFixedString.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -202,6 +203,13 @@ void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr } +void DataTypeFixedString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + const char * pos = reinterpret_cast(&static_cast(column).getChars()[n * row_num]); + protobuf.writeString(StringRef(pos, n)); +} + + MutableColumnPtr DataTypeFixedString::createColumn() const { return ColumnFixedString::create(n); diff --git a/dbms/src/DataTypes/DataTypeFixedString.h b/dbms/src/DataTypes/DataTypeFixedString.h index abea529ad42..c69d4859584 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.h +++ b/dbms/src/DataTypes/DataTypeFixedString.h @@ -64,6 +64,8 @@ public: void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.cpp b/dbms/src/DataTypes/DataTypeLowCardinality.cpp index e73deaae2ca..8d6c457646d 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinality.cpp @@ -729,10 +729,11 @@ void DataTypeLowCardinality::deserializeBinary(Field & field, ReadBuffer & istr) dictionary_type->deserializeBinary(field, istr); } -template +template void DataTypeLowCardinality::serializeImpl( - const IColumn & column, size_t row_num, WriteBuffer & ostr, - DataTypeLowCardinality::SerealizeFunctionPtr func, Args & ... args) const + const IColumn & column, size_t row_num, + DataTypeLowCardinality::SerializeFunctionPtr func, + OutputStream & ostr, Args & ... args) const { auto & low_cardinality_column = getColumnLowCardinality(column); size_t unique_row_number = low_cardinality_column.getIndexes().getUInt(row_num); @@ -741,8 +742,9 @@ void DataTypeLowCardinality::serializeImpl( template void DataTypeLowCardinality::deserializeImpl( - IColumn & column, ReadBuffer & istr, - DataTypeLowCardinality::DeserealizeFunctionPtr func, Args & ... args) const + IColumn & column, + DataTypeLowCardinality::DeserializeFunctionPtr func, + ReadBuffer & istr, Args & ... args) const { auto & low_cardinality_column= getColumnLowCardinality(column); auto temp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.h b/dbms/src/DataTypes/DataTypeLowCardinality.h index 5744419bf01..a099ce72fa8 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.h +++ b/dbms/src/DataTypes/DataTypeLowCardinality.h @@ -54,60 +54,65 @@ public: void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeBinary); + serializeImpl(column, row_num, &IDataType::serializeBinary, ostr); } void deserializeBinary(IColumn & column, ReadBuffer & istr) const override { - deserializeImpl(column, istr, &IDataType::deserializeBinary); + deserializeImpl(column, &IDataType::deserializeBinary, istr); } void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextEscaped, settings); + serializeImpl(column, row_num, &IDataType::serializeTextEscaped, ostr, settings); } void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - deserializeImpl(column, istr, &IDataType::deserializeTextEscaped, settings); + deserializeImpl(column, &IDataType::deserializeTextEscaped, istr, settings); } void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextQuoted, settings); + serializeImpl(column, row_num, &IDataType::serializeTextQuoted, ostr, settings); } void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - deserializeImpl(column, istr, &IDataType::deserializeTextQuoted, settings); + deserializeImpl(column, &IDataType::deserializeTextQuoted, istr, settings); } void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextCSV, settings); + serializeImpl(column, row_num, &IDataType::serializeTextCSV, ostr, settings); } void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - deserializeImpl(column, istr, &IDataType::deserializeTextCSV, settings); + deserializeImpl(column, &IDataType::deserializeTextCSV, istr, settings); } void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeText, settings); + serializeImpl(column, row_num, &IDataType::serializeText, ostr, settings); } void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextJSON, settings); + serializeImpl(column, row_num, &IDataType::serializeTextJSON, ostr, settings); } void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - deserializeImpl(column, istr, &IDataType::deserializeTextJSON, settings); + deserializeImpl(column, &IDataType::deserializeTextJSON, istr, settings); } void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextXML, settings); + serializeImpl(column, row_num, &IDataType::serializeTextXML, ostr, settings); + } + + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override + { + serializeImpl(column, row_num, &IDataType::serializeProtobuf, protobuf); } MutableColumnPtr createColumn() const override; @@ -143,19 +148,19 @@ public: private: - template - using SerealizeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, WriteBuffer &, Args & ...) const; + template + using SerializeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, OutputStream &, Args & ...) const; + + template + void serializeImpl(const IColumn & column, size_t row_num, SerializeFunctionPtr func, + OutputStream & ostr, Args & ... args) const; template - void serializeImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, - SerealizeFunctionPtr func, Args & ... args) const; + using DeserializeFunctionPtr = void (IDataType::*)(IColumn &, ReadBuffer &, Args & ...) const; template - using DeserealizeFunctionPtr = void (IDataType::*)(IColumn &, ReadBuffer &, Args & ...) const; - - template - void deserializeImpl(IColumn & column, ReadBuffer & istr, - DeserealizeFunctionPtr func, Args & ... args) const; + void deserializeImpl(IColumn & column, DeserializeFunctionPtr func, + ReadBuffer & istr, Args & ... args) const; template static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator); diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index 6322f5b2b17..f8c19b73bbe 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -311,6 +311,13 @@ void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, nested_data_type->serializeTextXML(col.getNestedColumn(), row_num, ostr, settings); } +void DataTypeNullable::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + const ColumnNullable & col = static_cast(column); + if (!col.isNullAt(row_num)) + nested_data_type->serializeProtobuf(col.getNestedColumn(), row_num, protobuf); +} + MutableColumnPtr DataTypeNullable::createColumn() const { return ColumnNullable::create(nested_data_type->createColumn(), ColumnUInt8::create()); diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index c6c228d441d..c45736d06a4 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -70,6 +70,8 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override { return Null(); } diff --git a/dbms/src/DataTypes/DataTypeNumberBase.cpp b/dbms/src/DataTypes/DataTypeNumberBase.cpp index 12904b34c17..94eb5618f3f 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.cpp +++ b/dbms/src/DataTypes/DataTypeNumberBase.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -201,6 +202,14 @@ void DataTypeNumberBase::deserializeBinaryBulk(IColumn & column, ReadBuffer & x.resize(initial_size + size / sizeof(typename ColumnVector::value_type)); } + +template +void DataTypeNumberBase::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeNumber(static_cast &>(column).getData()[row_num]); +} + + template MutableColumnPtr DataTypeNumberBase::createColumn() const { diff --git a/dbms/src/DataTypes/DataTypeNumberBase.h b/dbms/src/DataTypes/DataTypeNumberBase.h index 67da1c7602e..2728d32a6a9 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.h +++ b/dbms/src/DataTypes/DataTypeNumberBase.h @@ -29,7 +29,6 @@ public: Field getDefault() const override; /** Format is platform-dependent. */ - 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; @@ -37,6 +36,8 @@ public: void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; bool isParametric() const override { return false; } diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 55a4ac920b6..ed04b7600be 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include @@ -22,7 +23,6 @@ namespace DB { - void DataTypeString::serializeBinary(const Field & field, WriteBuffer & ostr) const { const String & s = get(field); @@ -304,6 +304,12 @@ void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, con } +void DataTypeString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeString(static_cast(column).getDataAt(row_num)); +} + + MutableColumnPtr DataTypeString::createColumn() const { return ColumnString::create(); diff --git a/dbms/src/DataTypes/DataTypeString.h b/dbms/src/DataTypes/DataTypeString.h index d0a210dcbf7..202c8374c27 100644 --- a/dbms/src/DataTypes/DataTypeString.h +++ b/dbms/src/DataTypes/DataTypeString.h @@ -45,6 +45,8 @@ public: void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override diff --git a/dbms/src/DataTypes/DataTypeTuple.cpp b/dbms/src/DataTypes/DataTypeTuple.cpp index 1f0b0f6ca3e..18ae891d576 100644 --- a/dbms/src/DataTypes/DataTypeTuple.cpp +++ b/dbms/src/DataTypes/DataTypeTuple.cpp @@ -407,6 +407,12 @@ void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } +void DataTypeTuple::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + for (const auto i : ext::range(0, ext::size(elems))) + elems[i]->serializeProtobuf(extractElementColumn(column, i), row_num, protobuf); +} + MutableColumnPtr DataTypeTuple::createColumn() const { size_t size = elems.size(); diff --git a/dbms/src/DataTypes/DataTypeTuple.h b/dbms/src/DataTypes/DataTypeTuple.h index 146efee936c..c3422be70db 100644 --- a/dbms/src/DataTypes/DataTypeTuple.h +++ b/dbms/src/DataTypes/DataTypeTuple.h @@ -77,6 +77,8 @@ public: DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override; diff --git a/dbms/src/DataTypes/DataTypeUUID.cpp b/dbms/src/DataTypes/DataTypeUUID.cpp index 8e022aaa6a0..9f913b5bf80 100644 --- a/dbms/src/DataTypes/DataTypeUUID.cpp +++ b/dbms/src/DataTypes/DataTypeUUID.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -71,6 +72,11 @@ void DataTypeUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const static_cast(column).getData().push_back(value); } +void DataTypeUUID::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeUUID(UUID(static_cast(column).getData()[row_num])); +} + bool DataTypeUUID::equals(const IDataType & rhs) const { diff --git a/dbms/src/DataTypes/DataTypeUUID.h b/dbms/src/DataTypes/DataTypeUUID.h index 3d43b947753..0a0ce6ad035 100644 --- a/dbms/src/DataTypes/DataTypeUUID.h +++ b/dbms/src/DataTypes/DataTypeUUID.h @@ -24,6 +24,7 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; bool canBeUsedInBitOperations() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index 0c5bd6c6559..439a98928ea 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -133,6 +134,13 @@ void DataTypeDecimal::deserializeBinaryBulk(IColumn & column, ReadBuffer & is } +template +void DataTypeDecimal::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeDecimal(static_cast(column).getData()[row_num], scale); +} + + template Field DataTypeDecimal::getDefault() const { diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index f192dfe1a75..3f93f5aaae1 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -100,6 +100,8 @@ public: void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + Field getDefault() const override; MutableColumnPtr createColumn() const override; bool equals(const IDataType & rhs) const override; diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 8f38e334024..fdac4e454bc 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -22,6 +22,8 @@ using MutableColumnPtr = COWPtr::MutablePtr; using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; +class ProtobufWriter; + /** Properties of data type. * Contains methods for serialization/deserialization. @@ -254,6 +256,9 @@ public: serializeText(column, row_num, ostr, settings); } + /** Serialize to a protobuf. */ + virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const = 0; + /** Create empty column for corresponding type. */ virtual MutableColumnPtr createColumn() const = 0; diff --git a/dbms/src/DataTypes/IDataTypeDummy.h b/dbms/src/DataTypes/IDataTypeDummy.h index 8f204b0e67c..bb122126577 100644 --- a/dbms/src/DataTypes/IDataTypeDummy.h +++ b/dbms/src/DataTypes/IDataTypeDummy.h @@ -28,6 +28,7 @@ public: void deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const override { throwNoSerialization(); } void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void serializeProtobuf(const IColumn &, size_t, ProtobufWriter &) const override { throwNoSerialization(); } MutableColumnPtr createColumn() const override { diff --git a/dbms/src/Formats/parseFormatSchema.cpp b/dbms/src/Formats/FormatSchemaInfo.cpp similarity index 100% rename from dbms/src/Formats/parseFormatSchema.cpp rename to dbms/src/Formats/FormatSchemaInfo.cpp diff --git a/dbms/src/Formats/parseFormatSchema.h b/dbms/src/Formats/FormatSchemaInfo.h similarity index 100% rename from dbms/src/Formats/parseFormatSchema.h rename to dbms/src/Formats/FormatSchemaInfo.h From 2886f42c0ed8df90832410c0022f5cd800f2d6a8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:42:20 +0300 Subject: [PATCH 040/324] Implemented a new block output stream using protobuf output format. --- dbms/src/Formats/FormatFactory.cpp | 2 + .../src/Formats/ProtobufBlockOutputStream.cpp | 75 +++++++++++++++++++ dbms/src/Formats/ProtobufBlockOutputStream.h | 47 ++++++++++++ 3 files changed, 124 insertions(+) create mode 100644 dbms/src/Formats/ProtobufBlockOutputStream.cpp create mode 100644 dbms/src/Formats/ProtobufBlockOutputStream.h diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 60672817db5..9674c376c47 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -111,6 +111,7 @@ void registerInputFormatTSKV(FormatFactory & factory); void registerOutputFormatTSKV(FormatFactory & factory); void registerInputFormatJSONEachRow(FormatFactory & factory); void registerOutputFormatJSONEachRow(FormatFactory & factory); +void registerOutputFormatProtobuf(FormatFactory & factory); /// Output only (presentational) formats. @@ -146,6 +147,7 @@ FormatFactory::FormatFactory() registerOutputFormatTSKV(*this); registerInputFormatJSONEachRow(*this); registerOutputFormatJSONEachRow(*this); + registerOutputFormatProtobuf(*this); registerInputFormatCapnProto(*this); registerOutputFormatPretty(*this); diff --git a/dbms/src/Formats/ProtobufBlockOutputStream.cpp b/dbms/src/Formats/ProtobufBlockOutputStream.cpp new file mode 100644 index 00000000000..985a92faaca --- /dev/null +++ b/dbms/src/Formats/ProtobufBlockOutputStream.cpp @@ -0,0 +1,75 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD; +} + + +ProtobufBlockOutputStream::ProtobufBlockOutputStream( + WriteBuffer & buffer_, + const Block & header_, + const google::protobuf::Descriptor * message_type, + const FormatSettings & format_settings_) + : writer(buffer_, message_type), header(header_), format_settings(format_settings_) +{ +} + +void ProtobufBlockOutputStream::write(const Block & block) +{ + std::vector columns_in_write_order; + const auto & fields_in_write_order = writer.fieldsInWriteOrder(); + columns_in_write_order.reserve(fields_in_write_order.size()); + for (size_t i = 0; i != fields_in_write_order.size(); ++i) + { + const auto * field = fields_in_write_order[i]; + const ColumnWithTypeAndName * column = nullptr; + if (block.has(field->name())) + { + column = &block.getByName(field->name()); + } + else if (field->is_required()) + { + throw Exception( + "Output doesn't have a column named '" + field->name() + "' which is required to write the output in the protobuf format.", + ErrorCodes::NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD); + } + columns_in_write_order.emplace_back(column); + } + + for (size_t row_num = 0; row_num != block.rows(); ++row_num) + { + writer.newMessage(); + for (const auto * column : columns_in_write_order) + { + if (column) + { + assert(column->name == writer.currentField()->name()); + column->type->serializeProtobuf(*(column->column), row_num, writer); + } + writer.nextField(); + } + } +} + + +void registerOutputFormatProtobuf(FormatFactory & factory) +{ + factory.registerOutputFormat( + "Protobuf", [](WriteBuffer & buf, const Block & header, const Context & context, const FormatSettings & format_settings) + { + const auto * message_type = ProtobufSchemas::instance().getMessageTypeForFormatSchema(FormatSchemaInfo(context, "proto")); + return std::make_shared(buf, header, message_type, format_settings); + }); +} + +} diff --git a/dbms/src/Formats/ProtobufBlockOutputStream.h b/dbms/src/Formats/ProtobufBlockOutputStream.h new file mode 100644 index 00000000000..c4b5236a5ab --- /dev/null +++ b/dbms/src/Formats/ProtobufBlockOutputStream.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include +#include +#include + + +namespace google +{ +namespace protobuf +{ + class Message; +} +} + + +namespace DB +{ +/** Stream designed to serialize data in the google protobuf format. + * Each row is written as a separated message. + * These messages are delimited according to documentation + * https://github.com/protocolbuffers/protobuf/blob/master/src/google/protobuf/util/delimited_message_util.h + * Serializing in the protobuf format requires the 'format_schema' setting to be set, e.g. + * SELECT * from table FORMAT Protobuf SETTINGS format_schema = 'schema:Message' + * where schema is the name of "schema.proto" file specifying protobuf schema. + */ +class ProtobufBlockOutputStream : public IBlockOutputStream +{ +public: + ProtobufBlockOutputStream( + WriteBuffer & buffer_, + const Block & header_, + const google::protobuf::Descriptor * message_prototype_, + const FormatSettings & format_settings_); + + Block getHeader() const override { return header; } + void write(const Block & block) override; + std::string getContentType() const override { return "application/octet-stream"; } + +private: + ProtobufWriter writer; + const Block header; + const FormatSettings format_settings; +}; + +} From 2e383bfdebdc873942bc11684310fa2e9871aa9c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:43:23 +0300 Subject: [PATCH 041/324] Added tests for writing protobufs. --- .../0_stateless/00825_protobuf_format.proto | 98 ++++++ .../00825_protobuf_format_output.reference | Bin 0 -> 2019 bytes .../00825_protobuf_format_output.sh | 60 ++++ .../00825_protobuf_format_syntax2.proto | 47 +++ utils/test-data-generator/CMakeLists.txt | 6 + .../ProtobufDelimitedMessagesSerializer.cpp | 332 ++++++++++++++++++ 6 files changed, 543 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00825_protobuf_format.proto create mode 100644 dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference create mode 100755 dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh create mode 100644 dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto create mode 100644 utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format.proto b/dbms/tests/queries/0_stateless/00825_protobuf_format.proto new file mode 100644 index 00000000000..47c4e74ff23 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format.proto @@ -0,0 +1,98 @@ +syntax = "proto3"; + +enum Gender { + female = 0; + male = 1; +}; + +enum ZodiacSign { + aries = 0; + taurus = 1; + gemini = 2; + cancer = 3; + leo = 4; + virgo = 5; + libra = 6; + scorpius = 7; + sagittarius = 8; + capricorn = 9; + aquarius = 10; + pisces = 11; +}; + +message Person { + string uuid = 1; + string name = 2; + string surname = 3; + Gender gender = 4; + uint32 birthDate = 5; + bytes photo = 6; + string phoneNumber = 7; + bool isOnline = 8; + fixed32 visitTime = 9; + uint32 age = 10; + ZodiacSign zodiacSign = 11; + repeated string songs = 12; + repeated uint32 color = 13; + string hometown = 14; + repeated float location = 15; + double pi = 16; + double lotteryWin = 17; + float someRatio = 18; + float temperature = 19; + sint64 randomBigNumber = 20; +}; + +enum OnlineStatus { + offline = 0; + online = 1; +}; + +message AltPerson { + enum Gender { + male = 0; + female = 1; + }; + repeated int32 location = 101 [packed=false]; + float pi = 103; + bytes uuid = 300; + bool newFieldBool = 299; + string name = 2; + Gender gender = 102; + int32 zodiacSign = 130; + int64 birthDate = 150; + bytes age = 111; + OnlineStatus isOnline = 1; + double someRatio = 100; + fixed64 visitTime = 15; + sfixed64 randomBigNumber = 140; + repeated int32 newFieldInt = 104; + repeated float color = 14; + uint64 lotteryWin = 202; + bytes surname = 10; + uint64 phoneNumber = 5; + sint32 temperature = 41; + string newFieldStr = 21; +}; + +message StrPerson { + string uuid = 1; + string name = 2; + string surname = 3; + string gender = 4; + string birthDate = 5; + string phoneNumber = 7; + string isOnline = 8; + string visitTime = 9; + string age = 10; + string zodiacSign = 11; + repeated string songs = 12; + repeated string color = 13; + string hometown = 14; + repeated string location = 15; + string pi = 16; + string lotteryWin = 17; + string someRatio = 18; + string temperature = 19; + string randomBigNumber = 20; +}; diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference new file mode 100644 index 0000000000000000000000000000000000000000..ed3e22f41e33b807ae354cd7045e96b86eae973f GIT binary patch literal 2019 zcmbtVe`p(39KV-KuXkCcT0xFPi)x$Dw2P0FR~ zXjD1}Djf=CV@NAwv*?IeSpRf?6b2nqWNrmT!T#e91i>Fn=s;T~zDtvC48a<|B$pTN z-F-jb&*$@f-;Y4xfTdH66>31$CJH1;U5A#fnNUkn({LP0Q^aiX_fA>4Rw3$4y7{Rf z=(yO1`H9?-&buDdiAfc#5~}H32=xECjt)m39+HyMA*V3uxVAN&Ys2U-&i9_WDoHfBc{nngyyCtC`Rl@Y&U?Pn-n{^Or*B!SO3! zy**P597t&kLd z>9?f!D?HhjVp$p-A(%p<8zwYNqQfLo*&ofZRh_g5gH~ZQJKfqCAGPy4RBcBvpB$fH zR1tzvl`4Z#75(Os!_oVP{K<|GF_7UhDHSboaoX z#5O}W%LwFtoQmk-Gi(*<@o@HCW(0k4?lkD`OB{?1bSHWT55gw{vp@tpx^=u+y7KNj zzF1X0w~6Cso?`E*#I&cFw_5WB-V@mG`T6-vT_r&;2@kCZt2ZN|Ey1T6bK)n`w<7o3 z*T3^@&$4u}ZN2GB@LkKE?DO)jCb3;jJQQnTRs%y43D=T`#HCkQa;+o;mj$pQtT*?1 z`LQ-9ZUC15a%<}PFXi&GbfJso-isaPJQ}L9bX@y%^ZKt%v6?y_h9b*9$-cqvX-|+H zCK@VCu=Q-HB=nSo_7!2HibS0gKW|`2ZmN>EcUjtwtT!EB-&o!BRoM7-qD6(pgemovziH!0uc==Mkk@22*sF=!&1tca7R;l zH}{V%%41410I|<*pGuU(;pV1>vEIt6$nD+_?9vatyE zDLT^7PAsz`_afA;M{f1)eqqF^)oNmhREUf)V0B<^SeO_sq?|&AR}jkZm|c7nRohyk zMAi+gsUBi1BSJA%PJl)au|myLpaIDU=@qnO5i~>g_-*lXz1@CG8IwY!Lu}m9^)+P- zPd60>5n{4bl)>sSpLO!Jvw+FqRYgd~r=Z)81W7YwlW}WcR+%azcKRBuvb(F5VU4QP zFm>}JXlA_Z2CJ3{WFWKv{7{wgB9IhHNacR~&_JSl2s1k_)n&(er$M?5(p dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e -o pipefail + +# Copy schema files to the current directory because the client can open schemas from the current directory only. +cp "$CURDIR/00825_protobuf_format.proto" 00825_protobuf_format_copy.proto +cp "$CURDIR/00825_protobuf_format_syntax2.proto" 00825_protobuf_format_syntax2_copy.proto + +# Run the client. +$CLICKHOUSE_CLIENT --multiquery <<'EOF' +SET allow_experimental_low_cardinality_type = 1; +CREATE DATABASE IF NOT EXISTS test; +DROP TABLE IF EXISTS test.table; + +CREATE TABLE test.table (uuid UUID, + name String, + surname String, + gender Enum8('male'=1, 'female'=0), + birthDate Date, + photo Nullable(String), + phoneNumber Nullable(FixedString(13)), + isOnline UInt8, + visitTime Nullable(DateTime), + age UInt8, + zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823, + 'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120, + 'pisces'=219), + songs Array(String), + color Array(UInt8), + hometown LowCardinality(String), + location Array(Decimal32(6)), + pi Nullable(Float64), + lotteryWin Nullable(Decimal64(2)), + someRatio Float32, + temperature Decimal32(1), + randomBigNumber Int64 + ) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO test.table VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000); +INSERT INTO test.table VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Tver', [54.782635, 32.045251], 3.14159, NULL, 0.007, 5.4, -20000000000000); +INSERT INTO test.table VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000); + +SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_copy:Person'; +SELECT 'ALTERNATIVE->'; +SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_copy:AltPerson'; +SELECT 'STRINGS->'; +SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_copy:StrPerson'; +SELECT 'SYNTAX2->'; +SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_syntax2_copy:Syntax2Person'; +EOF + +# Remove copies of the schema files. +rm "00825_protobuf_format_copy.proto" "00825_protobuf_format_syntax2_copy.proto" diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto b/dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto new file mode 100644 index 00000000000..dad1b596824 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto @@ -0,0 +1,47 @@ +syntax = "proto2"; + +message Syntax2Person { + enum Gender { + female = 0; + male = 1; + }; + + enum ZodiacSign { + aries = 0; + taurus = 1; + gemini = 2; + cancer = 3; + leo = 4; + virgo = 5; + libra = 6; + scorpius = 7; + sagittarius = 8; + capricorn = 9; + aquarius = 10; + pisces = 11; + }; + + required string uuid = 1; + required string name = 2; + required string surname = 3; + required Gender gender = 4; + required uint32 birthDate = 5; + optional bytes photo = 6; + optional string phoneNumber = 7; + optional bool isOnline = 8; + optional fixed32 visitTime = 9; + optional uint32 age = 10; + optional ZodiacSign zodiacSign = 11; + repeated string songs = 12; + repeated uint32 color = 13; + optional string hometown = 14 [default='Moscow']; + repeated float location = 15 [packed=true]; + optional double pi = 16; + optional double lotteryWin = 17; + optional float someRatio = 18; + optional float temperature = 19; + optional sint64 randomBigNumber = 20; + optional string newFieldStr = 21 [default='abc']; + optional int32 newFieldInt = 22 [default=-11]; + optional bool newBool = 23 [default=true]; +}; diff --git a/utils/test-data-generator/CMakeLists.txt b/utils/test-data-generator/CMakeLists.txt index 60f17894704..bf3cfa1bf7f 100644 --- a/utils/test-data-generator/CMakeLists.txt +++ b/utils/test-data-generator/CMakeLists.txt @@ -3,3 +3,9 @@ target_link_libraries(test-data-generator PRIVATE clickhouse_common_io ${Boost_P add_executable (markov-model markov-model.cpp) target_link_libraries(markov-model PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) + +protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs ProtobufDelimitedMessagesSerializer_Hdrs ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format.proto) +protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs2 ProtobufDelimitedMessagesSerializer_Hdrs2 ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto) +add_executable (ProtobufDelimitedMessagesSerializer ProtobufDelimitedMessagesSerializer.cpp ${ProtobufDelimitedMessagesSerializer_Srcs} ${ProtobufDelimitedMessagesSerializer_Hdrs} ${ProtobufDelimitedMessagesSerializer_Srcs2} ${ProtobufDelimitedMessagesSerializer_Hdrs2}) +target_include_directories (ProtobufDelimitedMessagesSerializer SYSTEM BEFORE PRIVATE ${Protobuf_INCLUDE_DIR} ${CMAKE_CURRENT_BINARY_DIR}) +target_link_libraries (ProtobufDelimitedMessagesSerializer PRIVATE ${Protobuf_LIBRARY}) \ No newline at end of file diff --git a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp new file mode 100644 index 00000000000..4a5596cb7f2 --- /dev/null +++ b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp @@ -0,0 +1,332 @@ +// Reference file generator for the test dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh + +#include +#include +#include "00825_protobuf_format.pb.h" +#include "00825_protobuf_format_syntax2.pb.h" + + +int main(int, char **) +{ + std::ostream* out = &std::cout; + + { + Person person; + person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); + person.set_name("Ivan"); + person.set_surname("Petrov"); + person.set_gender(Gender::male); + person.set_birthdate(4015); // 1980-12-29 + person.set_photo("png"); + person.set_phonenumber(std::string("+74951234567\0", 13)); // Converted from FixedString(13) + person.set_isonline(true); + person.set_visittime(1546703100); // 2019-01-05 18:45:00 + person.set_age(38); + person.set_zodiacsign(ZodiacSign::capricorn); + person.add_songs("Yesterday"); + person.add_songs("Flowers"); + person.add_color(255); + person.add_color(0); + person.add_color(0); + person.set_hometown("Moscow"); + person.add_location(55.753215); + person.add_location(37.622504); + person.set_pi(3.14); + person.set_lotterywin(214.10); + person.set_someratio(0.1); + person.set_temperature(5.8); + person.set_randombignumber(17060000000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + Person person; + person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); + person.set_name("Natalia"); + person.set_surname("Sokolova"); + person.set_gender(Gender::female); + person.set_birthdate(8102); // 1992-03-08 + person.set_photo("jpg"); + person.set_isonline(false); + person.set_age(26); + person.set_zodiacsign(ZodiacSign::pisces); + person.add_color(100); + person.add_color(200); + person.add_color(50); + person.set_hometown("Tver"); + person.add_location(54.782635); + person.add_location(32.045251); + person.set_pi(3.14159); + person.set_someratio(0.007); + person.set_temperature(5.4); + person.set_randombignumber(-20000000000000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + Person person; + person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); + person.set_name("Vasily"); + person.set_surname("Sidorov"); + person.set_gender(Gender::male); + person.set_birthdate(9339); // 1995-07-28 + person.set_photo("bmp"); + person.set_phonenumber("+442012345678"); + person.set_isonline(true); + person.set_visittime(1546117200); // 2018-12-30 00:00:00 + person.set_age(23); + person.set_zodiacsign(ZodiacSign::leo); + person.add_songs("Sunny"); + person.add_color(250); + person.add_color(244); + person.add_color(10); + person.set_hometown("Murmansk"); + person.add_location(68.970682); + person.add_location(33.074981); + person.set_pi(3.14159265358979); + person.set_lotterywin(100000000000); + person.set_someratio(800); + person.set_temperature(-3.2); + person.set_randombignumber(154400000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + *out << "ALTERNATIVE->" << std::endl; + + { + AltPerson person; + person.add_location(55); + person.add_location(37); + person.set_pi(3.14); + person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); + person.set_name("Ivan"); + person.set_gender(AltPerson::male); + person.set_zodiacsign(1222); // capricorn + person.set_birthdate(4015); // 1980-12-29 + person.set_age("38"); + person.set_isonline(OnlineStatus::online); + person.set_someratio(0.100000001490116119384765625); // 0.1 converted from float to double + person.set_visittime(1546703100); // 2019-01-05 18:45:00 + person.set_randombignumber(17060000000); + person.add_color(255); + person.add_color(0); + person.add_color(0); + person.set_lotterywin(214); + person.set_surname("Petrov"); + person.set_phonenumber(+74951234567); + person.set_temperature(5); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + AltPerson person; + person.add_location(54); + person.add_location(32); + person.set_pi(3.14159); + person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); + person.set_name("Natalia"); + person.set_gender(AltPerson::female); + person.set_zodiacsign(219); // pisces + person.set_birthdate(8102); // 1992-03-08 + person.set_age("26"); + person.set_isonline(OnlineStatus::offline); + person.set_someratio(0.007000000216066837310791015625); // 0.007 converted from float to double + person.set_randombignumber(-20000000000000); + person.add_color(100); + person.add_color(200); + person.add_color(50); + person.set_surname("Sokolova"); + person.set_temperature(5); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + AltPerson person; + person.add_location(68); + person.add_location(33); + person.set_pi(3.1415926535897); + person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); + person.set_name("Vasily"); + person.set_gender(AltPerson::male); + person.set_zodiacsign(723); // leo + person.set_birthdate(9339); // 1995-07-28 + person.set_age("23"); + person.set_isonline(OnlineStatus::online); + person.set_someratio(800); + person.set_visittime(1546117200); // 2018-12-30 00:00:00 + person.set_randombignumber(154400000); + person.add_color(250); + person.add_color(244); + person.add_color(10); + person.set_lotterywin(100000000000); + person.set_surname("Sidorov"); + person.set_phonenumber(+442012345678); + person.set_temperature(-3); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + *out << "STRINGS->" << std::endl; + + { + StrPerson person; + person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); + person.set_name("Ivan"); + person.set_surname("Petrov"); + person.set_gender("male"); + person.set_birthdate("1980-12-29"); + person.set_phonenumber(std::string("+74951234567\0", 13)); // Converted from FixedString(13) + person.set_isonline("1"); + person.set_visittime("2019-01-05 18:45:00"); + person.set_age("38"); + person.set_zodiacsign("capricorn"); + person.add_songs("Yesterday"); + person.add_songs("Flowers"); + person.add_color("255"); + person.add_color("0"); + person.add_color("0"); + person.set_hometown("Moscow"); + person.add_location("55.753215"); + person.add_location("37.622504"); + person.set_pi("3.14"); + person.set_lotterywin("214.10"); + person.set_someratio("0.1"); + person.set_temperature("5.8"); + person.set_randombignumber("17060000000"); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + StrPerson person; + person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); + person.set_name("Natalia"); + person.set_surname("Sokolova"); + person.set_gender("female"); + person.set_birthdate("1992-03-08"); + person.set_isonline("0"); + person.set_age("26"); + person.set_zodiacsign("pisces"); + person.add_color("100"); + person.add_color("200"); + person.add_color("50"); + person.set_hometown("Tver"); + person.add_location("54.782635"); + person.add_location("32.045251"); + person.set_pi("3.14159"); + person.set_someratio("0.007"); + person.set_temperature("5.4"); + person.set_randombignumber("-20000000000000"); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + StrPerson person; + person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); + person.set_name("Vasily"); + person.set_surname("Sidorov"); + person.set_gender("male"); + person.set_birthdate("1995-07-28"); + person.set_phonenumber("+442012345678"); + person.set_isonline("1"); + person.set_visittime("2018-12-30 00:00:00"); + person.set_age("23"); + person.set_zodiacsign("leo"); + person.add_songs("Sunny"); + person.add_color("250"); + person.add_color("244"); + person.add_color("10"); + person.set_hometown("Murmansk"); + person.add_location("68.970682"); + person.add_location("33.074981"); + person.set_pi("3.14159265358979"); + person.set_lotterywin("100000000000.00"); + person.set_someratio("800"); + person.set_temperature("-3.2"); + person.set_randombignumber("154400000"); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + *out << "SYNTAX2->" << std::endl; + + { + Syntax2Person person; + person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); + person.set_name("Ivan"); + person.set_surname("Petrov"); + person.set_gender(Syntax2Person::male); + person.set_birthdate(4015); // 1980-12-29 + person.set_photo("png"); + person.set_phonenumber(std::string("+74951234567\0", 13)); // Converted from FixedString(13) + person.set_isonline(true); + person.set_visittime(1546703100); // 2019-01-05 18:45:00 + person.set_age(38); + person.set_zodiacsign(Syntax2Person::capricorn); + person.add_songs("Yesterday"); + person.add_songs("Flowers"); + person.add_color(255); + person.add_color(0); + person.add_color(0); + person.set_hometown("Moscow"); + person.add_location(55.753215); + person.add_location(37.622504); + person.set_pi(3.14); + person.set_lotterywin(214.10); + person.set_someratio(0.1); + person.set_temperature(5.8); + person.set_randombignumber(17060000000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + Syntax2Person person; + person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); + person.set_name("Natalia"); + person.set_surname("Sokolova"); + person.set_gender(Syntax2Person::female); + person.set_birthdate(8102); // 1992-03-08 + person.set_photo("jpg"); + person.set_isonline(false); + person.set_age(26); + person.set_zodiacsign(Syntax2Person::pisces); + person.add_color(100); + person.add_color(200); + person.add_color(50); + person.set_hometown("Tver"); + person.add_location(54.782635); + person.add_location(32.045251); + person.set_pi(3.14159); + person.set_someratio(0.007); + person.set_temperature(5.4); + person.set_randombignumber(-20000000000000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + Syntax2Person person; + person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); + person.set_name("Vasily"); + person.set_surname("Sidorov"); + person.set_gender(Syntax2Person::male); + person.set_birthdate(9339); // 1995-07-28 + person.set_photo("bmp"); + person.set_phonenumber("+442012345678"); + person.set_isonline(true); + person.set_visittime(1546117200); // 2018-12-30 00:00:00 + person.set_age(23); + person.set_zodiacsign(Syntax2Person::leo); + person.add_songs("Sunny"); + person.add_color(250); + person.add_color(244); + person.add_color(10); + person.set_hometown("Murmansk"); + person.add_location(68.970682); + person.add_location(33.074981); + person.set_pi(3.14159265358979); + person.set_lotterywin(100000000000); + person.set_someratio(800); + person.set_temperature(-3.2); + person.set_randombignumber(154400000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + return 0; +} From af81d3cf0dc212911348ee3eff583ed6f881ec36 Mon Sep 17 00:00:00 2001 From: George G Date: Thu, 24 Jan 2019 14:55:59 +0000 Subject: [PATCH 042/324] docs: Fix `sudo` position in CPU scaling governor command (#4142) --- docs/en/operations/tips.md | 2 +- docs/ru/operations/tips.md | 2 +- docs/zh/operations/tips.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index e3b66a639ba..3508c66f1af 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -21,7 +21,7 @@ You can use `turbostat` to view the CPU's actual clock rate under a load. Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. ```bash -sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor +echo 'performance' | sudo tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor ``` ## CPU Limitations diff --git a/docs/ru/operations/tips.md b/docs/ru/operations/tips.md index a66a760bc79..e9bbf77d041 100644 --- a/docs/ru/operations/tips.md +++ b/docs/ru/operations/tips.md @@ -21,7 +21,7 @@ Turbo-Boost крайне не рекомендуется отключать. П Нужно всегда использовать `performance` scaling governor. `ondemand` scaling governor работает намного хуже при постоянно высоком спросе. ```bash -sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor +echo 'performance' | sudo tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor ``` ## Ограничение CPU diff --git a/docs/zh/operations/tips.md b/docs/zh/operations/tips.md index 11b7a782deb..1dcbb74a2cc 100644 --- a/docs/zh/operations/tips.md +++ b/docs/zh/operations/tips.md @@ -21,7 +21,7 @@ You can use `turbostat` to view the CPU's actual clock rate under a load. Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. ```bash -sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor +echo 'performance' | sudo tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor ``` ## CPU Limitations From 5f1007a8df32890a1ded0c40dd787757d5c91fbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 24 Jan 2019 17:56:04 +0300 Subject: [PATCH 043/324] Refactor Join. --- dbms/src/Common/ColumnsHashing.h | 86 ++++++++++++---- dbms/src/Common/ColumnsHashingImpl.h | 12 +-- dbms/src/Interpreters/Join.cpp | 146 ++++++++++++++++----------- dbms/src/Interpreters/Join.h | 1 + 4 files changed, 158 insertions(+), 87 deletions(-) diff --git a/dbms/src/Common/ColumnsHashing.h b/dbms/src/Common/ColumnsHashing.h index 61c3d71cbbc..c84bf95ce53 100644 --- a/dbms/src/Common/ColumnsHashing.h +++ b/dbms/src/Common/ColumnsHashing.h @@ -36,10 +36,10 @@ using HashMethodContextPtr = std::shared_ptr; /// For the case where there is one numeric key. -template /// UInt8/16/32/64 for any type with corresponding bit width. -struct HashMethodOneNumber : public columns_hashing_impl::HashMethodBase +template /// UInt8/16/32/64 for any type with corresponding bit width. +struct HashMethodOneNumber : public columns_hashing_impl::HashMethodBase { - using Base = columns_hashing_impl::HashMethodBase; + using Base = columns_hashing_impl::HashMethodBase; const char * vec; /// If the keys of a fixed length then key_sizes contains their lengths, empty otherwise. @@ -90,10 +90,10 @@ protected: /// For the case where there is one string key. -template -struct HashMethodString : public columns_hashing_impl::HashMethodBase +template +struct HashMethodString : public columns_hashing_impl::HashMethodBase { - using Base = columns_hashing_impl::HashMethodBase; + using Base = columns_hashing_impl::HashMethodBase; const IColumn::Offset * offsets; const UInt8 * chars; @@ -107,7 +107,7 @@ struct HashMethodString : public columns_hashing_impl::HashMethodBase ALWAYS_INLINE typename Base::EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool) @@ -150,10 +150,10 @@ protected: /// For the case where there is one fixed-length string key. -template -struct HashMethodFixedString : public columns_hashing_impl::HashMethodBase +template +struct HashMethodFixedString : public columns_hashing_impl::HashMethodBase { - using Base = columns_hashing_impl::HashMethodBase; + using Base = columns_hashing_impl::HashMethodBase; size_t n; const ColumnFixedString::Chars * chars; @@ -396,10 +396,13 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod if (is_nullable && row == 0) { visit_cache[row] = VisitValue::Found; + bool has_null_key = data.hasNullKeyData(); + data.hasNullKeyData() = true; + if constexpr (has_mapped) - return EmplaceResult(data.getNullKeyData(), mapped_cache[0], !data.hasNullKeyData()); + return EmplaceResult(data.getNullKeyData(), mapped_cache[0], !has_null_key); else - return EmplaceResult(!data.hasNullKeyData()); + return EmplaceResult(!has_null_key); } if (visit_cache[row] == VisitValue::Found) @@ -446,7 +449,7 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod if (is_nullable && row == 0) { if constexpr (has_mapped) - return FindResult(data.hasNullKeyData() ? data.getNullKeyData() : Mapped(), data.hasNullKeyData()); + return FindResult(data.hasNullKeyData() ? &data.getNullKeyData() : nullptr, data.hasNullKeyData()); else return FindResult(data.hasNullKeyData()); } @@ -454,7 +457,7 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod if (visit_cache[row] != VisitValue::Empty) { if constexpr (has_mapped) - return FindResult(mapped_cache[row], visit_cache[row] == VisitValue::Found); + return FindResult(&mapped_cache[row], visit_cache[row] == VisitValue::Found); else return FindResult(visit_cache[row] == VisitValue::Found); } @@ -477,7 +480,7 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod } if constexpr (has_mapped) - return FindResult(mapped_cache[row], found); + return FindResult(&mapped_cache[row], found); else return FindResult(found); } @@ -507,10 +510,10 @@ template <> struct LowCardinalityKeys {}; /// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits. -template +template struct HashMethodKeysFixed : private columns_hashing_impl::BaseStateKeysFixed - , public columns_hashing_impl::HashMethodBase + , public columns_hashing_impl::HashMethodBase { static constexpr bool has_nullable_keys = has_nullable_keys_; static constexpr bool has_low_cardinality = has_low_cardinality_; @@ -520,7 +523,7 @@ struct HashMethodKeysFixed size_t keys_size; using Base = columns_hashing_impl::BaseStateKeysFixed; - using BaseHashed = columns_hashing_impl::HashMethodBase; + using BaseHashed = columns_hashing_impl::HashMethodBase; HashMethodKeysFixed(const ColumnRawPtrs & key_columns, const Sizes & key_sizes, const HashMethodContextPtr &) : key_sizes(std::move(key_sizes)), keys_size(key_columns.size()) @@ -590,10 +593,10 @@ struct HashMethodKeysFixed * That is, for example, for strings, it contains first the serialized length of the string, and then the bytes. * Therefore, when aggregating by several strings, there is no ambiguity. */ -template -struct HashMethodSerialized : public columns_hashing_impl::HashMethodBase +template +struct HashMethodSerialized : public columns_hashing_impl::HashMethodBase { - using Base = columns_hashing_impl::HashMethodBase; + using Base = columns_hashing_impl::HashMethodBase; ColumnRawPtrs key_columns; size_t keys_size; @@ -641,5 +644,46 @@ protected: } }; +/// For the case where there is one string key. +template +struct HashMethodHashed : public columns_hashing_impl::HashMethodBase +{ + using Base = columns_hashing_impl::HashMethodBase; + using Key = UInt128; + + ColumnRawPtrs key_columns; + + HashMethodHashed(ColumnRawPtrs key_columns, const Sizes &, const HashMethodContextPtr &) + : key_columns(std::move(key_columns)) {} + + static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; } + + UInt128 getKey(size_t row) const { return hash128(row, key_columns.size(), key_columns); } + + template + ALWAYS_INLINE typename Base::EmplaceResult emplaceKey(Data & data, size_t row, Arena & /*pool*/) + { + typename Data::iterator it; + return Base::emplaceKeyImpl(getKey(row), data, it); + } + + template + ALWAYS_INLINE typename Base::FindResult findKey(Data & data, size_t row, Arena & /*pool*/) + { + return Base::findKeyImpl(getKey(row), data); + } + + template + ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & /*pool*/) + { + return data.hash(getKey(row)); + } + + static StringRef getValueRef(const Value & value) + { + return StringRef(reinterpret_cast(&value.first), sizeof(value.first)); + } +}; + } } diff --git a/dbms/src/Common/ColumnsHashingImpl.h b/dbms/src/Common/ColumnsHashingImpl.h index 565940b3338..542971d4cca 100644 --- a/dbms/src/Common/ColumnsHashingImpl.h +++ b/dbms/src/Common/ColumnsHashingImpl.h @@ -44,7 +44,7 @@ public: : value(value), cached_value(cached_value), inserted(inserted) {} bool isInserted() const { return inserted; } - const auto & getMapped() const { return value; } + auto & getMapped() const { return value; } void setMapped(const Mapped & mapped) { value = cached_value = mapped; } }; @@ -61,13 +61,13 @@ public: template class FindResultImpl { - Mapped value; + Mapped * value; bool found; public: - FindResultImpl(Mapped value, bool found) : value(value), found(found) {} + FindResultImpl(Mapped * value, bool found) : value(value), found(found) {} bool isFound() const { return found; } - const Mapped & getMapped() const { return value; } + Mapped & getMapped() const { return *value; } }; template <> @@ -142,7 +142,7 @@ protected: if (cache.check(key)) { if constexpr (has_mapped) - return FindResult(cache.found ? cache.value.second : Mapped(), cache.found); + return FindResult(&cache.value.second, cache.found); else return FindResult(cache.found); } @@ -168,7 +168,7 @@ protected: } if constexpr (has_mapped) - return FindResult(found ? it->second : Mapped(), found); + return FindResult(found ? &it->second : nullptr, found); else return FindResult(found); } diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 8783d16c3c1..62f5ea2be1d 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -150,18 +150,54 @@ static size_t getTotalByteCountImpl(const Maps & maps, Join::Type type) } -template -struct KeyGetterForType; +template +struct KeyGetterForTypeImpl; -template <> struct KeyGetterForType { using Type = JoinKeyGetterOneNumber; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterOneNumber; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterOneNumber; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterOneNumber; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterString; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterFixedString; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterFixed; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterFixed; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterHashed; }; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodOneNumber; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodOneNumber; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodOneNumber; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodOneNumber; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodString; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodFixedString; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodKeysFixed; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodKeysFixed; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodHashed; +}; + +template +struct KeyGetterForType +{ + using Value = typename Data::value_type; + using Mapped_t = typename Data::mapped_type; + using Mapped = std::conditional_t, const Mapped_t, Mapped_t>; + using Type = typename KeyGetterForTypeImpl::Type; +}; /// Do I need to use the hash table maps_*_full, in which we remember whether the row was joined. @@ -309,40 +345,30 @@ namespace template struct Inserter { - static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool); + static void insert(Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool); }; template struct Inserter { - static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool) + static void insert(Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { - typename Map::iterator it; - bool inserted; - map.emplace(key, it, inserted); + auto emplace_result = key_getter.emplaceKey(map, i, pool); - if (inserted) - { - KeyGetter::onNewKey(it->first, pool); - new (&it->second) typename Map::mapped_type(stored_block, i); - } + if (emplace_result.isInserted()) + new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); } }; template struct Inserter { - static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool) + static void insert(Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { - typename Map::iterator it; - bool inserted; - map.emplace(key, it, inserted); + auto emplace_result = key_getter.emplaceKey(map, i, pool); - if (inserted) - { - KeyGetter::onNewKey(it->first, pool); - new (&it->second) typename Map::mapped_type(stored_block, i); - } + if (emplace_result.isInserted()) + new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); else { /** The first element of the list is stored in the value of the hash table, the rest in the pool. @@ -350,9 +376,10 @@ namespace * That is, the former second element, if it was, will be the third, and so on. */ auto elem = pool.alloc(); + auto & mapped = emplace_result.getMapped(); - elem->next = it->second.next; - it->second.next = elem; + elem->next = mapped.next; + mapped.next = elem; elem->block = stored_block; elem->row_num = i; } @@ -363,17 +390,16 @@ namespace template void NO_INLINE insertFromBlockImplTypeCase( Map & map, size_t rows, const ColumnRawPtrs & key_columns, - size_t keys_size, const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) { - KeyGetter key_getter(key_columns); + KeyGetter key_getter(key_columns, key_sizes, nullptr); for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) continue; - auto key = key_getter.getKey(key_columns, keys_size, i, key_sizes); - Inserter::insert(map, key, stored_block, i, pool); + Inserter::insert(map, key_getter, stored_block, i, pool); } } @@ -381,19 +407,19 @@ namespace template void insertFromBlockImplType( Map & map, size_t rows, const ColumnRawPtrs & key_columns, - size_t keys_size, const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) { if (null_map) - insertFromBlockImplTypeCase(map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + insertFromBlockImplTypeCase(map, rows, key_columns, key_sizes, stored_block, null_map, pool); else - insertFromBlockImplTypeCase(map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + insertFromBlockImplTypeCase(map, rows, key_columns, key_sizes, stored_block, null_map, pool); } template void insertFromBlockImpl( Join::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns, - size_t keys_size, const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) { switch (type) { @@ -402,8 +428,8 @@ namespace #define M(TYPE) \ case Join::Type::TYPE: \ - insertFromBlockImplType::Type>(\ - *maps.TYPE, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); \ + insertFromBlockImplType>::Type>(\ + *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, pool); \ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -486,16 +512,16 @@ bool Join::insertFromBlock(const Block & block) if (!getFullness(kind)) { if (strictness == ASTTableJoin::Strictness::Any) - insertFromBlockImpl(type, maps_any, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + insertFromBlockImpl(type, maps_any, rows, key_columns, key_sizes, stored_block, null_map, pool); else - insertFromBlockImpl(type, maps_all, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + insertFromBlockImpl(type, maps_all, rows, key_columns, key_sizes, stored_block, null_map, pool); } else { if (strictness == ASTTableJoin::Strictness::Any) - insertFromBlockImpl(type, maps_any_full, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + insertFromBlockImpl(type, maps_any_full, rows, key_columns, key_sizes, stored_block, null_map, pool); else - insertFromBlockImpl(type, maps_all_full, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + insertFromBlockImpl(type, maps_all_full, rows, key_columns, key_sizes, stored_block, null_map, pool); } } @@ -511,14 +537,14 @@ namespace template struct Adder { - static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, + static void addFound(const typename Map::mapped_type & mapped, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & right_indexes) { (*filter)[i] = 1; for (size_t j = 0; j < num_columns_to_add; ++j) - added_columns[j]->insertFrom(*it->second.block->getByPosition(right_indexes[j]).column.get(), it->second.row_num); + added_columns[j]->insertFrom(*mapped.block->getByPosition(right_indexes[j]).column, mapped.row_num); } static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, @@ -534,14 +560,14 @@ namespace template struct Adder { - static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, + static void addFound(const typename Map::mapped_type & mapped, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & right_indexes) { (*filter)[i] = 1; for (size_t j = 0; j < num_columns_to_add; ++j) - added_columns[j]->insertFrom(*it->second.block->getByPosition(right_indexes[j]).column.get(), it->second.row_num); + added_columns[j]->insertFrom(*mapped.block->getByPosition(right_indexes[j]).column, mapped.row_num); } static void addNotFound(size_t /*num_columns_to_add*/, MutableColumns & /*added_columns*/, @@ -554,14 +580,14 @@ namespace template struct Adder { - static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, + static void addFound(const typename Map::mapped_type & mapped, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets, const std::vector & right_indexes) { (*filter)[i] = 1; size_t rows_joined = 0; - for (auto current = &static_cast(it->second); current != nullptr; current = current->next) + for (auto current = &static_cast(mapped); current != nullptr; current = current->next) { for (size_t j = 0; j < num_columns_to_add; ++j) added_columns[j]->insertFrom(*current->block->getByPosition(right_indexes[j]).column.get(), current->row_num); @@ -600,10 +626,10 @@ namespace IColumn::Offset & current_offset, std::unique_ptr & offsets_to_replicate, const std::vector & right_indexes) { - size_t keys_size = key_columns.size(); size_t num_columns_to_add = right_indexes.size(); - KeyGetter key_getter(key_columns); + Arena pool; + KeyGetter key_getter(key_columns, key_sizes, nullptr); for (size_t i = 0; i < rows; ++i) { @@ -614,14 +640,14 @@ namespace } else { - auto key = key_getter.getKey(key_columns, keys_size, i, key_sizes); - typename Map::const_iterator it = map.find(key); + auto find_result = key_getter.findKey(map, i, pool); - if (it != map.end()) + if (find_result.isFound()) { - it->second.setUsed(); + auto & mapped = find_result.getMapped(); + mapped.setUsed(); Adder::addFound( - it, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get(), right_indexes); + mapped, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get(), right_indexes); } else Adder::addNotFound( @@ -748,7 +774,7 @@ void Join::joinBlockImpl( { #define M(TYPE) \ case Join::Type::TYPE: \ - joinBlockImplType::Type>(\ + joinBlockImplType>::Type>(\ *maps.TYPE, rows, key_columns, key_sizes, added_columns, null_map, \ filter, current_offset, offsets_to_replicate, right_indexes); \ break; diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 65371239851..3b3a6dc3a22 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -8,6 +8,7 @@ #include #include +#include #include #include From ccd9bd0877ecdab2a23f3578f2cfce7006aec14e Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 24 Jan 2019 18:06:15 +0300 Subject: [PATCH 044/324] fix build --- dbms/src/Interpreters/QueryNormalizer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index aa00e6e8b5c..a5151be7c64 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -376,7 +376,7 @@ void QueryNormalizer::extractJoinUsingColumns(const ASTPtr ast, Data & data) for (const auto & key : keys.children) if (auto opt_column = getIdentifierName(key)) data.join_using_columns.insert(*opt_column); - else if (auto * literal = typeid_cast(key.get())) + else if (typeid_cast(key.get())) data.join_using_columns.insert(key->getColumnName()); else { From b8d7922e42f0f82f348a2c113671ea2c19e4a93f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Jan 2019 19:10:05 +0300 Subject: [PATCH 045/324] CLICKHOUSE-4170: Add docs for datasets downloading (#4144) * CLICKHOUSE-4170: Add docs for datasets downloading * CLICKHOUSE-4170: Fix Review remarks --- .../example_datasets/nyc_taxi.md | 20 ++++++++++++++++ .../example_datasets/ontime.md | 23 ++++++++++++++++++- .../example_datasets/nyc_taxi.md | 21 ++++++++++++++++- .../example_datasets/ontime.md | 23 ++++++++++++++++++- 4 files changed, 84 insertions(+), 3 deletions(-) diff --git a/docs/en/getting_started/example_datasets/nyc_taxi.md b/docs/en/getting_started/example_datasets/nyc_taxi.md index 0f5f1cb3cb4..ed073fc85a6 100644 --- a/docs/en/getting_started/example_datasets/nyc_taxi.md +++ b/docs/en/getting_started/example_datasets/nyc_taxi.md @@ -1,5 +1,10 @@ # New York Taxi Data +This dataset can be obtained in two ways: + +- import from raw data +- download of prepared partitions + ## How to Import The Raw Data See and for the description of a dataset and instructions for downloading. @@ -272,6 +277,21 @@ WHERE (table = 'trips_mergetree') AND active Among other things, you can run the OPTIMIZE query on MergeTree. But it's not required, since everything will be fine without it. +## Dowload of Prepared Partitions + +```bash +curl -O https://clickhouse-datasets.s3.yandex.net/trips_mergetree/partitions/trips_mergetree.tar +tar xvf trips_mergetree.tar -C /var/lib/clickhouse # path to ClickHouse data directory +# check permissions of unpacked data, fix if required +sudo service clickhouse-server restart +clickhouse-client --query "select count(*) from datasets.trips_mergetree" +``` + +!!!info + If you will run queries described below, you have to use full table name, + `datasets.trips_mergetree`. + + ## Results on Single Server Q1: diff --git a/docs/en/getting_started/example_datasets/ontime.md b/docs/en/getting_started/example_datasets/ontime.md index c4dcaf9575e..6cf2e1ecb52 100644 --- a/docs/en/getting_started/example_datasets/ontime.md +++ b/docs/en/getting_started/example_datasets/ontime.md @@ -1,6 +1,13 @@ # OnTime +This dataset can be obtained in two ways: + +- import from raw data +- download of prepared partitions + +## Import From Raw Data + Downloading data: ```bash @@ -137,7 +144,21 @@ Loading data: for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done ``` -Queries: +## Dowload of Prepared Partitions + +```bash +curl -O https://clickhouse-datasets.s3.yandex.net/ontime/partitions/ontime.tar +tar xvf ontime.tar -C /var/lib/clickhouse # path to ClickHouse data directory +# check permissions of unpacked data, fix if required +sudo service clickhouse-server restart +clickhouse-client --query "select count(*) from datasets.ontime" +``` + +!!!info + If you will run queries described below, you have to use full table name, + `datasets.ontime`. + +## Queries Q0. diff --git a/docs/ru/getting_started/example_datasets/nyc_taxi.md b/docs/ru/getting_started/example_datasets/nyc_taxi.md index cd86ff05b76..1050a589669 100644 --- a/docs/ru/getting_started/example_datasets/nyc_taxi.md +++ b/docs/ru/getting_started/example_datasets/nyc_taxi.md @@ -1,5 +1,10 @@ # Данные о такси в Нью-Йорке +Этот датасет может быть получен двумя способами: + +- импорт из сырых данных; +- скачивание готовых партиций. + ## Как импортировать сырые данные См. и для описания набора данных и инструкций по загрузке. @@ -272,7 +277,21 @@ WHERE (table = 'trips_mergetree') AND active Между прочим, на MergeTree можно запустить запрос OPTIMIZE. Но это не обязательно, всё будет в порядке и без этого. -## Results on single server +## Скачивание готовых партиций + +```bash +curl -O https://clickhouse-datasets.s3.yandex.net/trips_mergetree/partitions/trips_mergetree.tar +tar xvf trips_mergetree.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse +# убедитесь, что установлены корректные права доступа на файлы +sudo service clickhouse-server restart +clickhouse-client --query "select count(*) from datasets.trips_mergetree" +``` + +!!!info + Если вы собираетесь выполнять запросы, приведенные ниже, то к имени таблицы + нужно добавить имя базы, `datasets.trips_mergetree`. + +## Результаты на одном сервере Q1: diff --git a/docs/ru/getting_started/example_datasets/ontime.md b/docs/ru/getting_started/example_datasets/ontime.md index f138817b400..89a3fa7ebe6 100644 --- a/docs/ru/getting_started/example_datasets/ontime.md +++ b/docs/ru/getting_started/example_datasets/ontime.md @@ -1,6 +1,13 @@ # OnTime +Этот датасет может быть получен двумя способами: + +- импорт из сырых данных; +- скачивание готовых партиций. + +## Импорт из сырых данных + Скачивание данных: ```bash @@ -137,7 +144,21 @@ CREATE TABLE `ontime` ( for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done ``` -Запросы: +## Скачивание готовых партиций + +```bash +curl -O https://clickhouse-datasets.s3.yandex.net/ontime/partitions/ontime.tar +tar xvf ontime.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse +# убедитесь, что установлены корректные права доступа на файлы +sudo service clickhouse-server restart +clickhouse-client --query "select count(*) from datasets.ontime" +``` + +!!!info + Если вы собираетесь выполнять запросы, приведенные ниже, то к имени таблицы + нужно добавить имя базы, `datasets.ontime`. + +## Запросы: Q0. From d79ed9fde68827d216b6bbabf738bbcc640e6412 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 24 Jan 2019 19:34:27 +0300 Subject: [PATCH 046/324] Update 00502_sum_map.reference --- dbms/tests/queries/0_stateless/00502_sum_map.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.reference b/dbms/tests/queries/0_stateless/00502_sum_map.reference index 67639ad230f..7bb325be814 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.reference +++ b/dbms/tests/queries/0_stateless/00502_sum_map.reference @@ -9,7 +9,7 @@ 2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10] 2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10] ([1],[10]) -([1,4,8], [10,20,10]) +([1,4,8],[10,20,10]) ([1],[1]) ([1],[1]) (['a'],[1]) From 0e59fe0b46e0e12b9e652e995a7939ad75d624bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 20:12:05 +0300 Subject: [PATCH 047/324] Renamed setting and variables #3973 --- dbms/src/Interpreters/Join.cpp | 22 +++++++++---------- dbms/src/Interpreters/Join.h | 14 ++++++------ dbms/src/Interpreters/Settings.h | 2 +- dbms/src/Storages/StorageJoin.cpp | 8 +++---- .../0_stateless/00830_join_overwrite.sql | 2 +- 5 files changed, 24 insertions(+), 24 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 553e5e8810a..6ef873fb6c7 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -34,11 +34,11 @@ namespace ErrorCodes Join::Join(const Names & key_names_right_, bool use_nulls_, const SizeLimits & limits, - ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool overwrite_) + ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool any_take_last_row_) : kind(kind_), strictness(strictness_), key_names_right(key_names_right_), use_nulls(use_nulls_), - overwrite(overwrite_), + any_take_last_row(any_take_last_row_), log(&Logger::get("Join")), limits(limits) { @@ -480,7 +480,7 @@ bool Join::insertFromBlock(const Block & block) namespace { - template + template struct Adder; template @@ -526,8 +526,8 @@ namespace } }; - template - struct Adder + template + struct Adder { static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets, @@ -553,7 +553,7 @@ namespace { (*filter)[i] = 0; - if (!pad_left) + if (!fill_left) { (*offsets)[i] = current_offset; } @@ -584,7 +584,7 @@ namespace { if (has_null_map && (*null_map)[i]) { - Adder::pad_left, STRICTNESS, Map>::addNotFound( + Adder::fill_left, STRICTNESS, Map>::addNotFound( num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get()); } else @@ -595,11 +595,11 @@ namespace if (it != map.end()) { it->second.setUsed(); - Adder::pad_left, STRICTNESS, Map>::addFound( + Adder::fill_left, STRICTNESS, Map>::addFound( it, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get(), right_indexes); } else - Adder::pad_left, STRICTNESS, Map>::addNotFound( + Adder::fill_left, STRICTNESS, Map>::addNotFound( num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get()); } } @@ -889,7 +889,7 @@ void Join::joinGetImpl(Block & block, const String & column_name, const Maps & m // TODO: support composite key -// TODO: return multible columns as named tuple +// TODO: return multiple columns as named tuple // TODO: return array of values when strictness == ASTTableJoin::Strictness::All void Join::joinGet(Block & block, const String & column_name) const { @@ -902,7 +902,7 @@ void Join::joinGet(Block & block, const String & column_name) const if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any) { - if (overwrite) + if (any_take_last_row) joinGetImpl(block, column_name, std::get(maps)); else joinGetImpl(block, column_name, std::get(maps)); diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 43de3092409..3a70f1d07ac 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -223,7 +223,7 @@ class Join { public: Join(const Names & key_names_right_, bool use_nulls_, const SizeLimits & limits, - ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool overwrite_ = false); + ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool any_take_last_row_ = false); bool empty() { return type == Type::EMPTY; } @@ -380,18 +380,18 @@ public: struct KindTrait { // Affects the Adder trait so that when the right part is empty, adding a default value on the left - static constexpr bool pad_left = static_in_v; + static constexpr bool fill_left = static_in_v; // Affects the Map trait so that a `used` flag is attached to map slots in order to // generate default values on the right when the left part is empty - static constexpr bool pad_right = static_in_v; + static constexpr bool fill_right = static_in_v; }; - template + template struct MapGetterImpl; template - using Map = typename MapGetterImpl::pad_right, strictness, overwrite>::Map; + using Map = typename MapGetterImpl::fill_right, strictness, overwrite>::Map; static constexpr std::array STRICTNESSES = {ASTTableJoin::Strictness::Any, ASTTableJoin::Strictness::All}; static constexpr std::array KINDS @@ -402,7 +402,7 @@ public: template bool dispatch(Func && func) { - if (overwrite) + if (any_take_last_row) { return static_for<0, KINDS.size()>([&](auto i) { @@ -464,7 +464,7 @@ private: bool use_nulls; /// Overwrite existing values when encountering the same key again - bool overwrite; + bool any_take_last_row; /** Blocks of "right" table. */ diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 464cbad8a8e..b719a11b1a6 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -253,7 +253,7 @@ struct Settings M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).") \ M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).") \ M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ - M(SettingBool, join_overwrite, false, "Whether to overwrite existing values when encountering the same key again.") \ + M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.") \ \ M(SettingUInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ M(SettingUInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index dfaea34905c..78ef15a4fc8 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -134,7 +134,7 @@ void registerStorageJoin(StorageFactory & factory) auto max_rows_in_join = settings.max_rows_in_join; auto max_bytes_in_join = settings.max_bytes_in_join; auto join_overflow_mode = settings.join_overflow_mode; - auto join_overwrite = settings.join_overwrite; + auto join_any_take_last_row = settings.join_any_take_last_row; if (args.storage_def && args.storage_def->settings) { @@ -148,8 +148,8 @@ void registerStorageJoin(StorageFactory & factory) max_bytes_in_join.set(setting.value); else if (setting.name == "join_overflow_mode") join_overflow_mode.set(setting.value); - else if (setting.name == "join_overwrite") - join_overwrite.set(setting.value); + else if (setting.name == "join_any_take_last_row") + join_any_take_last_row.set(setting.value); else throw Exception( "Unknown setting " + setting.name + " for storage " + args.engine_name, @@ -166,7 +166,7 @@ void registerStorageJoin(StorageFactory & factory) kind, strictness, args.columns, - join_overwrite); + join_any_take_last_row); }); } diff --git a/dbms/tests/queries/0_stateless/00830_join_overwrite.sql b/dbms/tests/queries/0_stateless/00830_join_overwrite.sql index 1f981eb00bd..d6fca471746 100644 --- a/dbms/tests/queries/0_stateless/00830_join_overwrite.sql +++ b/dbms/tests/queries/0_stateless/00830_join_overwrite.sql @@ -6,7 +6,7 @@ CREATE TABLE kv (k UInt32, v UInt32) ENGINE Join(Any, Left, k); INSERT INTO kv VALUES (1, 2); INSERT INTO kv VALUES (1, 3); SELECT joinGet('kv', 'v', toUInt32(1)); -CREATE TABLE kv_overwrite (k UInt32, v UInt32) ENGINE Join(Any, Left, k) SETTINGS join_overwrite = 1; +CREATE TABLE kv_overwrite (k UInt32, v UInt32) ENGINE Join(Any, Left, k) SETTINGS join_any_take_last_row = 1; INSERT INTO kv_overwrite VALUES (1, 2); INSERT INTO kv_overwrite VALUES (1, 3); SELECT joinGet('kv_overwrite', 'v', toUInt32(1)); From ffd18163de014126b04a06f97c8c214172b1c45a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 21:33:57 +0300 Subject: [PATCH 048/324] Fixed fuzz test in UBSan: added parameter check for quantile*Weighted function --- dbms/src/AggregateFunctions/AggregateFunctionQuantile.h | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h index ad25ff95af3..cee2b6fe0c0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -100,7 +100,7 @@ public: return res; } - void NO_SANITIZE_UNDEFINED add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { /// Out of range conversion may occur. This is Ok. @@ -177,8 +177,11 @@ public: static void assertSecondArg(const DataTypes & argument_types) { if constexpr (has_second_arg) - /// TODO: check that second argument is of numerical type. + { assertBinary(Name::name, argument_types); + if (!isUnsignedInteger(argument_types[1])) + throw Exception("Second argument (weight) for function " + std::string(Name::name) + " must be unsigned integer, but it has type " + argument_types[1]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } else assertUnary(Name::name, argument_types); } From 8a11f753bef9841ff1418e759c96100def3c727b Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 24 Jan 2019 21:34:31 +0300 Subject: [PATCH 049/324] fix test results --- .../0_stateless/00050_any_left_join.reference | 10 +++++----- .../0_stateless/00050_any_left_join.sql | 6 +++--- .../00051_any_inner_join.reference | 10 +++++----- .../0_stateless/00051_any_inner_join.sql | 6 +++--- .../00053_all_inner_join.reference | 20 +++++++++---------- .../0_stateless/00053_all_inner_join.sql | 6 +++--- .../queries/0_stateless/00054_join_string.sql | 6 +++--- 7 files changed, 32 insertions(+), 32 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00050_any_left_join.reference b/dbms/tests/queries/0_stateless/00050_any_left_join.reference index dc46f7e8a75..98dc95d69e1 100644 --- a/dbms/tests/queries/0_stateless/00050_any_left_join.reference +++ b/dbms/tests/queries/0_stateless/00050_any_left_join.reference @@ -1,10 +1,10 @@ 0 0 0 -1 1 0 +1 0 0 2 2 1 -3 3 0 +3 0 0 4 4 2 -5 5 0 +5 0 0 6 6 3 -7 7 0 +7 0 0 8 8 4 -9 9 0 +9 0 0 diff --git a/dbms/tests/queries/0_stateless/00050_any_left_join.sql b/dbms/tests/queries/0_stateless/00050_any_left_join.sql index 2440a9f93ab..61ccb200307 100644 --- a/dbms/tests/queries/0_stateless/00050_any_left_join.sql +++ b/dbms/tests/queries/0_stateless/00050_any_left_join.sql @@ -1,9 +1,9 @@ -SELECT * FROM +SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 -) +) AS a ANY LEFT JOIN ( SELECT number * 2 AS k, number AS joined FROM system.numbers LIMIT 10 -) +) AS b USING k; diff --git a/dbms/tests/queries/0_stateless/00051_any_inner_join.reference b/dbms/tests/queries/0_stateless/00051_any_inner_join.reference index 2a61b78f6ea..a7949c6a350 100644 --- a/dbms/tests/queries/0_stateless/00051_any_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00051_any_inner_join.reference @@ -1,5 +1,5 @@ -0 0 -2 1 -4 2 -6 3 -8 4 +0 0 0 +2 2 1 +4 4 2 +6 6 3 +8 8 4 diff --git a/dbms/tests/queries/0_stateless/00051_any_inner_join.sql b/dbms/tests/queries/0_stateless/00051_any_inner_join.sql index a3ff3c437cf..986c798d763 100644 --- a/dbms/tests/queries/0_stateless/00051_any_inner_join.sql +++ b/dbms/tests/queries/0_stateless/00051_any_inner_join.sql @@ -1,9 +1,9 @@ -SELECT * FROM +SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 -) +) AS a ANY INNER JOIN ( SELECT number * 2 AS k, number AS joined FROM system.numbers LIMIT 10 -) +) AS b USING k; diff --git a/dbms/tests/queries/0_stateless/00053_all_inner_join.reference b/dbms/tests/queries/0_stateless/00053_all_inner_join.reference index 41707378e8d..15bed0fbe0c 100644 --- a/dbms/tests/queries/0_stateless/00053_all_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00053_all_inner_join.reference @@ -1,10 +1,10 @@ -0 0 -0 1 -1 2 -1 3 -2 4 -2 5 -3 6 -3 7 -4 8 -4 9 +0 0 0 +0 1 1 +1 2 2 +1 3 3 +2 4 4 +2 0 5 +3 0 6 +3 0 7 +4 0 8 +4 0 9 diff --git a/dbms/tests/queries/0_stateless/00053_all_inner_join.sql b/dbms/tests/queries/0_stateless/00053_all_inner_join.sql index 18a94588033..b2e65c80574 100644 --- a/dbms/tests/queries/0_stateless/00053_all_inner_join.sql +++ b/dbms/tests/queries/0_stateless/00053_all_inner_join.sql @@ -1,9 +1,9 @@ -SELECT * FROM +SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 -) +) AS a ALL INNER JOIN ( SELECT intDiv(number, 2) AS k, number AS joined FROM system.numbers LIMIT 10 -) +) AS b USING k; diff --git a/dbms/tests/queries/0_stateless/00054_join_string.sql b/dbms/tests/queries/0_stateless/00054_join_string.sql index 7912a08c1d6..00e1278d330 100644 --- a/dbms/tests/queries/0_stateless/00054_join_string.sql +++ b/dbms/tests/queries/0_stateless/00054_join_string.sql @@ -1,9 +1,9 @@ -SELECT * FROM +SELECT a.*, b.* FROM ( SELECT reinterpretAsString(number + reinterpretAsUInt8('A')) AS k FROM system.numbers LIMIT 10 -) +) AS a ALL LEFT JOIN ( SELECT reinterpretAsString(intDiv(number, 2) + reinterpretAsUInt8('A')) AS k, number AS joined FROM system.numbers LIMIT 10 -) +) AS b USING k; From 2d490acc3e8a5f70bbdbe3be524bb3388d25a887 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 21:36:45 +0300 Subject: [PATCH 050/324] Added test --- .../00831_quantile_weighted_parameter_check.reference | 1 + .../0_stateless/00831_quantile_weighted_parameter_check.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.reference create mode 100644 dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.sql diff --git a/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.reference b/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.reference new file mode 100644 index 00000000000..7f8f011eb73 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.reference @@ -0,0 +1 @@ +7 diff --git a/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.sql b/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.sql new file mode 100644 index 00000000000..1d31b80f193 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.sql @@ -0,0 +1,2 @@ +SELECT quantileExactWeighted(0.5)(number, number) FROM numbers(10); +SELECT quantileExactWeighted(0.5)(number, 0.1) FROM numbers(10); -- { serverError 43 } From 1e04ef16a07f00f32fffd96a475c1ef4e6668511 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 24 Jan 2019 21:55:18 +0300 Subject: [PATCH 051/324] one more test fix --- dbms/tests/queries/0_stateless/00054_join_string.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00054_join_string.sql b/dbms/tests/queries/0_stateless/00054_join_string.sql index 00e1278d330..7912a08c1d6 100644 --- a/dbms/tests/queries/0_stateless/00054_join_string.sql +++ b/dbms/tests/queries/0_stateless/00054_join_string.sql @@ -1,9 +1,9 @@ -SELECT a.*, b.* FROM +SELECT * FROM ( SELECT reinterpretAsString(number + reinterpretAsUInt8('A')) AS k FROM system.numbers LIMIT 10 -) AS a +) ALL LEFT JOIN ( SELECT reinterpretAsString(intDiv(number, 2) + reinterpretAsUInt8('A')) AS k, number AS joined FROM system.numbers LIMIT 10 -) AS b +) USING k; From 04fa827e4b26f872e482a25bb273dfda816ca606 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 22:21:47 +0300 Subject: [PATCH 052/324] Style --- dbms/src/Common/SpaceSaving.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/SpaceSaving.h b/dbms/src/Common/SpaceSaving.h index feba4414a54..6e446d1c938 100644 --- a/dbms/src/Common/SpaceSaving.h +++ b/dbms/src/Common/SpaceSaving.h @@ -79,7 +79,7 @@ private: constexpr uint64_t nextAlphaSize(uint64_t x) { constexpr uint64_t ALPHA_MAP_ELEMENTS_PER_COUNTER = 6; - return 1ULL<<(sizeof(uint64_t) * 8 - __builtin_clzll(x * ALPHA_MAP_ELEMENTS_PER_COUNTER)); + return 1ULL << (sizeof(uint64_t) * 8 - __builtin_clzll(x * ALPHA_MAP_ELEMENTS_PER_COUNTER)); } public: From 854817c232ddf60f6f22eb87420e3b5a82705465 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 22:22:26 +0300 Subject: [PATCH 053/324] Fixed -Wshadow-field #4005 --- dbms/programs/client/Client.cpp | 4 ++-- dbms/src/Interpreters/executeQuery.cpp | 4 ++-- dbms/src/Parsers/ASTQueryWithOutput.cpp | 12 ++++++------ dbms/src/Parsers/ASTQueryWithOutput.h | 2 +- dbms/src/Parsers/ParserQueryWithOutput.cpp | 4 ++-- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 4c69d5d3e6a..74b1db69896 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1210,9 +1210,9 @@ private: const auto & id = typeid_cast(*query_with_output->format); current_format = id.name; } - if (query_with_output->settings) + if (query_with_output->settings_ast) { - InterpreterSetQuery(query_with_output->settings, context).executeForCurrentContext(); + InterpreterSetQuery(query_with_output->settings_ast, context).executeForCurrentContext(); } } diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index a2a6bb46035..6a21437399b 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -499,8 +499,8 @@ void executeQuery( ? *getIdentifierName(ast_query_with_output->format) : context.getDefaultFormat(); - if (ast_query_with_output && ast_query_with_output->settings) - InterpreterSetQuery(ast_query_with_output->settings, context).executeForCurrentContext(); + if (ast_query_with_output && ast_query_with_output->settings_ast) + InterpreterSetQuery(ast_query_with_output->settings_ast, context).executeForCurrentContext(); BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader()); diff --git a/dbms/src/Parsers/ASTQueryWithOutput.cpp b/dbms/src/Parsers/ASTQueryWithOutput.cpp index fd6957c436b..c28b15cb8c1 100644 --- a/dbms/src/Parsers/ASTQueryWithOutput.cpp +++ b/dbms/src/Parsers/ASTQueryWithOutput.cpp @@ -15,10 +15,10 @@ void ASTQueryWithOutput::cloneOutputOptions(ASTQueryWithOutput & cloned) const cloned.format = format->clone(); cloned.children.push_back(cloned.format); } - if (settings) + if (settings_ast) { - cloned.settings = settings->clone(); - cloned.children.push_back(cloned.settings); + cloned.settings_ast = settings_ast->clone(); + cloned.children.push_back(cloned.settings_ast); } } @@ -40,10 +40,10 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat format->formatImpl(s, state, frame); } - if (settings) + if (settings_ast) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SETTINGS " << (s.hilite ? hilite_none : ""); - settings->formatImpl(s, state, frame); + settings_ast->formatImpl(s, state, frame); } } @@ -53,7 +53,7 @@ bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast) { ast_with_output->format.reset(); ast_with_output->out_file.reset(); - ast_with_output->settings.reset(); + ast_with_output->settings_ast.reset(); return true; } diff --git a/dbms/src/Parsers/ASTQueryWithOutput.h b/dbms/src/Parsers/ASTQueryWithOutput.h index ec525b3aca9..9018d5661d9 100644 --- a/dbms/src/Parsers/ASTQueryWithOutput.h +++ b/dbms/src/Parsers/ASTQueryWithOutput.h @@ -14,7 +14,7 @@ class ASTQueryWithOutput : public IAST public: ASTPtr out_file; ASTPtr format; - ASTPtr settings; + ASTPtr settings_ast; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const final; diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index 830060cb953..91e320deeeb 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -87,9 +87,9 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (s_settings.ignore(pos, expected)) { ParserSetQuery parser_settings(true); - if (!parser_settings.parse(pos, query_with_output.settings, expected)) + if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) return false; - query_with_output.children.push_back(query_with_output.settings); + query_with_output.children.push_back(query_with_output.settings_ast); } if (explain_ast) From a28f0acf09c448b2fc2f531170cf0ec0ef309895 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 22:49:36 +0300 Subject: [PATCH 054/324] Removed non-breaking spaces: find dbms -name '*.h' -or -name '*.cpp' | xargs grep -l $'\xc2\xa0' | xargs sed -i -r -e 's/\xc2\xa0/ /' --- .../AggregateFunctions/UniqCombinedBiasData.h | 38 ++++++------- dbms/src/AggregateFunctions/UniquesHashSet.h | 54 +++++++++---------- dbms/src/Client/tests/test_connect.cpp | 6 +-- dbms/src/Common/InterruptListener.h | 8 +-- 4 files changed, 53 insertions(+), 53 deletions(-) diff --git a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h index 7206aabcf7e..36d4c5fb99e 100644 --- a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h +++ b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h @@ -6,25 +6,25 @@ namespace DB { /** Data for HyperLogLogBiasEstimator in the uniqCombined function. -  * The development plan is as follows: -  * 1. Assemble ClickHouse. -  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() -  * and another array for getBiases(). -  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. -  * 4. Assemble ClickHouse. -  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: -  * - raw_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) -  * - linear_counting_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) -  * - bias_corrected_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) -  * 6. Generate a graph with gnuplot based on this data. -  * 7. Determine the minimum number of unique values at which it is better to correct the error -  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. -  * 7. Accordingly, update the constant in the function getThreshold() -  * 8. Assemble ClickHouse. -  */ +  * The development plan is as follows: +  * 1. Assemble ClickHouse. +  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() +  * and another array for getBiases(). +  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. +  * 4. Assemble ClickHouse. +  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: +  * - raw_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) +  * - linear_counting_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) +  * - bias_corrected_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) +  * 6. Generate a graph with gnuplot based on this data. +  * 7. Determine the minimum number of unique values at which it is better to correct the error +  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. +  * 7. Accordingly, update the constant in the function getThreshold() +  * 8. Assemble ClickHouse. +  */ struct UniqCombinedBiasData { using InterpolatedData = std::array; diff --git a/dbms/src/AggregateFunctions/UniquesHashSet.h b/dbms/src/AggregateFunctions/UniquesHashSet.h index b0036d4832d..6f3bdc4e7f4 100644 --- a/dbms/src/AggregateFunctions/UniquesHashSet.h +++ b/dbms/src/AggregateFunctions/UniquesHashSet.h @@ -15,33 +15,33 @@ /** Approximate calculation of anything, as usual, is constructed according to the following scheme: -  * - some data structure is used to calculate the value of X; -  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); -  * - after processing all elements, the data structure is in some state S; -  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: -  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. -  */ +  * - some data structure is used to calculate the value of X; +  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); +  * - after processing all elements, the data structure is in some state S; +  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: +  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. +  */ /** In particular, what is described below can be found by the name of the BJKST algorithm. -  */ +  */ /** Very simple hash-set for approximate number of unique values. -  * Works like this: -  * - you can insert UInt64; -  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; -  * - the original value is not saved (lost); -  * - further all operations are made with these hashes; -  * - hash table is constructed according to the scheme: -  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); -  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); -  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; -  * - buffer growth by 2 times when filling more than 50%; -  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, -  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; -  * - if the situation repeats, then only elements dividing by 4, etc., are taken. -  * - the size() method returns an approximate number of elements that have been inserted into the set; -  * - there are methods for quick reading and writing in binary and text form. -  */ +  * Works like this: +  * - you can insert UInt64; +  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; +  * - the original value is not saved (lost); +  * - further all operations are made with these hashes; +  * - hash table is constructed according to the scheme: +  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); +  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); +  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; +  * - buffer growth by 2 times when filling more than 50%; +  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, +  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; +  * - if the situation repeats, then only elements dividing by 4, etc., are taken. +  * - the size() method returns an approximate number of elements that have been inserted into the set; +  * - there are methods for quick reading and writing in binary and text form. +  */ /// The maximum degree of buffer size before the values are discarded #define UNIQUES_HASH_MAX_SIZE_DEGREE 17 @@ -50,8 +50,8 @@ #define UNIQUES_HASH_MAX_SIZE (1ULL << (UNIQUES_HASH_MAX_SIZE_DEGREE - 1)) /** The number of least significant bits used for thinning. The remaining high-order bits are used to determine the position in the hash table. -  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) -  */ +  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) +  */ #define UNIQUES_HASH_BITS_FOR_SKIP (32 - UNIQUES_HASH_MAX_SIZE_DEGREE) /// Initial buffer size degree @@ -59,8 +59,8 @@ /** This hash function is not the most optimal, but UniquesHashSet states counted with it, -  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. -  */ +  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. +  */ struct UniquesHashSetDefaultHash { size_t operator() (UInt64 x) const diff --git a/dbms/src/Client/tests/test_connect.cpp b/dbms/src/Client/tests/test_connect.cpp index 75eb606cc97..50075cc24a6 100644 --- a/dbms/src/Client/tests/test_connect.cpp +++ b/dbms/src/Client/tests/test_connect.cpp @@ -9,9 +9,9 @@ /** In a loop it connects to the server and immediately breaks the connection. -  * Using the SO_LINGER option, we ensure that the connection is terminated by sending a RST packet (not FIN). -  * This behavior causes a bug in the TCPServer implementation in the Poco library. -  */ + * Using the SO_LINGER option, we ensure that the connection is terminated by sending a RST packet (not FIN). + * This behavior causes a bug in the TCPServer implementation in the Poco library. + */ int main(int argc, char ** argv) try { diff --git a/dbms/src/Common/InterruptListener.h b/dbms/src/Common/InterruptListener.h index eb818671037..bcb7c18c6fb 100644 --- a/dbms/src/Common/InterruptListener.h +++ b/dbms/src/Common/InterruptListener.h @@ -45,10 +45,10 @@ static int sigtimedwait(const sigset_t *set, siginfo_t *info, const struct times /** As long as there exists an object of this class - it blocks the INT signal, at the same time it lets you know if it came. -  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. -  * Use only one instance of this class at a time. -  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. -  */ +  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. +  * Use only one instance of this class at a time. +  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. +  */ class InterruptListener { private: From 304c7b03861cca72d9a0cee56468c8f54e5bb414 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 22:50:16 +0300 Subject: [PATCH 055/324] Style #4005 --- dbms/src/Formats/ProtobufWriter.cpp | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/dbms/src/Formats/ProtobufWriter.cpp b/dbms/src/Formats/ProtobufWriter.cpp index d17754e3478..1f2d200a7ca 100644 --- a/dbms/src/Formats/ProtobufWriter.cpp +++ b/dbms/src/Formats/ProtobufWriter.cpp @@ -21,8 +21,6 @@ namespace ErrorCodes } -// Converter ----------------------------------------------------------------------------------------------------------- - class ProtobufWriter::Converter : private boost::noncopyable { public: @@ -128,8 +126,6 @@ protected: }; -// ToStringConverter --------------------------------------------------------------------------------------------------- - class ProtobufWriter::ToStringConverter : public Converter { public: @@ -232,8 +228,6 @@ private: }; -// ToNumberConverter --------------------------------------------------------------------------------------------------- - template class ProtobufWriter::ToNumberConverter : public Converter { @@ -396,8 +390,6 @@ private: }; -// ToBoolConverter ----------------------------------------------------------------------------------------------------- - class ProtobufWriter::ToBoolConverter : public Converter { public: @@ -451,8 +443,6 @@ private: }; -// ToEnumConverter ----------------------------------------------------------------------------------------------------- - class ProtobufWriter::ToEnumConverter : public Converter { public: @@ -552,8 +542,6 @@ private: }; -// ProtobufWriter ------------------------------------------------------------------------------------------------------ - ProtobufWriter::ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type) : simple_writer(out) { enumerateFieldsInWriteOrder(message_type); From 5ccc42fd96fcf177ebb2c81596216e6e7f46a1ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Jan 2019 01:59:45 +0300 Subject: [PATCH 056/324] Revert "Removed non-breaking spaces: find dbms -name '*.h' -or -name '*.cpp' | xargs grep -l $'\xc2\xa0' | xargs sed -i -r -e 's/\xc2\xa0/ /'" This reverts commit a28f0acf09c448b2fc2f531170cf0ec0ef309895. --- .../AggregateFunctions/UniqCombinedBiasData.h | 38 ++++++------- dbms/src/AggregateFunctions/UniquesHashSet.h | 54 +++++++++---------- dbms/src/Client/tests/test_connect.cpp | 6 +-- dbms/src/Common/InterruptListener.h | 8 +-- 4 files changed, 53 insertions(+), 53 deletions(-) diff --git a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h index 36d4c5fb99e..7206aabcf7e 100644 --- a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h +++ b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h @@ -6,25 +6,25 @@ namespace DB { /** Data for HyperLogLogBiasEstimator in the uniqCombined function. -  * The development plan is as follows: -  * 1. Assemble ClickHouse. -  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() -  * and another array for getBiases(). -  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. -  * 4. Assemble ClickHouse. -  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: -  * - raw_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) -  * - linear_counting_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) -  * - bias_corrected_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) -  * 6. Generate a graph with gnuplot based on this data. -  * 7. Determine the minimum number of unique values at which it is better to correct the error -  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. -  * 7. Accordingly, update the constant in the function getThreshold() -  * 8. Assemble ClickHouse. -  */ +  * The development plan is as follows: +  * 1. Assemble ClickHouse. +  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() +  * and another array for getBiases(). +  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. +  * 4. Assemble ClickHouse. +  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: +  * - raw_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) +  * - linear_counting_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) +  * - bias_corrected_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) +  * 6. Generate a graph with gnuplot based on this data. +  * 7. Determine the minimum number of unique values at which it is better to correct the error +  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. +  * 7. Accordingly, update the constant in the function getThreshold() +  * 8. Assemble ClickHouse. +  */ struct UniqCombinedBiasData { using InterpolatedData = std::array; diff --git a/dbms/src/AggregateFunctions/UniquesHashSet.h b/dbms/src/AggregateFunctions/UniquesHashSet.h index 6f3bdc4e7f4..b0036d4832d 100644 --- a/dbms/src/AggregateFunctions/UniquesHashSet.h +++ b/dbms/src/AggregateFunctions/UniquesHashSet.h @@ -15,33 +15,33 @@ /** Approximate calculation of anything, as usual, is constructed according to the following scheme: -  * - some data structure is used to calculate the value of X; -  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); -  * - after processing all elements, the data structure is in some state S; -  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: -  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. -  */ +  * - some data structure is used to calculate the value of X; +  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); +  * - after processing all elements, the data structure is in some state S; +  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: +  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. +  */ /** In particular, what is described below can be found by the name of the BJKST algorithm. -  */ +  */ /** Very simple hash-set for approximate number of unique values. -  * Works like this: -  * - you can insert UInt64; -  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; -  * - the original value is not saved (lost); -  * - further all operations are made with these hashes; -  * - hash table is constructed according to the scheme: -  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); -  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); -  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; -  * - buffer growth by 2 times when filling more than 50%; -  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, -  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; -  * - if the situation repeats, then only elements dividing by 4, etc., are taken. -  * - the size() method returns an approximate number of elements that have been inserted into the set; -  * - there are methods for quick reading and writing in binary and text form. -  */ +  * Works like this: +  * - you can insert UInt64; +  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; +  * - the original value is not saved (lost); +  * - further all operations are made with these hashes; +  * - hash table is constructed according to the scheme: +  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); +  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); +  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; +  * - buffer growth by 2 times when filling more than 50%; +  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, +  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; +  * - if the situation repeats, then only elements dividing by 4, etc., are taken. +  * - the size() method returns an approximate number of elements that have been inserted into the set; +  * - there are methods for quick reading and writing in binary and text form. +  */ /// The maximum degree of buffer size before the values are discarded #define UNIQUES_HASH_MAX_SIZE_DEGREE 17 @@ -50,8 +50,8 @@ #define UNIQUES_HASH_MAX_SIZE (1ULL << (UNIQUES_HASH_MAX_SIZE_DEGREE - 1)) /** The number of least significant bits used for thinning. The remaining high-order bits are used to determine the position in the hash table. -  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) -  */ +  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) +  */ #define UNIQUES_HASH_BITS_FOR_SKIP (32 - UNIQUES_HASH_MAX_SIZE_DEGREE) /// Initial buffer size degree @@ -59,8 +59,8 @@ /** This hash function is not the most optimal, but UniquesHashSet states counted with it, -  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. -  */ +  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. +  */ struct UniquesHashSetDefaultHash { size_t operator() (UInt64 x) const diff --git a/dbms/src/Client/tests/test_connect.cpp b/dbms/src/Client/tests/test_connect.cpp index 50075cc24a6..75eb606cc97 100644 --- a/dbms/src/Client/tests/test_connect.cpp +++ b/dbms/src/Client/tests/test_connect.cpp @@ -9,9 +9,9 @@ /** In a loop it connects to the server and immediately breaks the connection. - * Using the SO_LINGER option, we ensure that the connection is terminated by sending a RST packet (not FIN). - * This behavior causes a bug in the TCPServer implementation in the Poco library. - */ +  * Using the SO_LINGER option, we ensure that the connection is terminated by sending a RST packet (not FIN). +  * This behavior causes a bug in the TCPServer implementation in the Poco library. +  */ int main(int argc, char ** argv) try { diff --git a/dbms/src/Common/InterruptListener.h b/dbms/src/Common/InterruptListener.h index bcb7c18c6fb..eb818671037 100644 --- a/dbms/src/Common/InterruptListener.h +++ b/dbms/src/Common/InterruptListener.h @@ -45,10 +45,10 @@ static int sigtimedwait(const sigset_t *set, siginfo_t *info, const struct times /** As long as there exists an object of this class - it blocks the INT signal, at the same time it lets you know if it came. -  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. -  * Use only one instance of this class at a time. -  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. -  */ +  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. +  * Use only one instance of this class at a time. +  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. +  */ class InterruptListener { private: From ef50601b5ceeeaf5763eab6c0013954c12eb00b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Jan 2019 02:01:56 +0300 Subject: [PATCH 057/324] Removed non-breaking spaces: find dbms -name '*.h' -or -name '*.cpp' | xargs grep -l $'\xc2\xa0' | xargs sed -i -r -e 's/\xc2\xa0/ /g' --- .../AggregateFunctions/UniqCombinedBiasData.h | 38 ++++++------- dbms/src/AggregateFunctions/UniquesHashSet.h | 54 +++++++++---------- dbms/src/Client/tests/test_connect.cpp | 6 +-- dbms/src/Common/InterruptListener.h | 8 +-- 4 files changed, 53 insertions(+), 53 deletions(-) diff --git a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h index 7206aabcf7e..0a69a211206 100644 --- a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h +++ b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h @@ -6,25 +6,25 @@ namespace DB { /** Data for HyperLogLogBiasEstimator in the uniqCombined function. -  * The development plan is as follows: -  * 1. Assemble ClickHouse. -  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() -  * and another array for getBiases(). -  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. -  * 4. Assemble ClickHouse. -  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: -  * - raw_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) -  * - linear_counting_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) -  * - bias_corrected_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) -  * 6. Generate a graph with gnuplot based on this data. -  * 7. Determine the minimum number of unique values at which it is better to correct the error -  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. -  * 7. Accordingly, update the constant in the function getThreshold() -  * 8. Assemble ClickHouse. -  */ + * The development plan is as follows: + * 1. Assemble ClickHouse. + * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() + * and another array for getBiases(). + * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. + * 4. Assemble ClickHouse. + * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: + * - raw_graph.txt (1st column: the present number of unique values; + * 2nd column: relative error in the case of HyperLogLog without applying any corrections) + * - linear_counting_graph.txt (1st column: the present number of unique values; + * 2nd column: relative error in the case of HyperLogLog using LinearCounting) + * - bias_corrected_graph.txt (1st column: the present number of unique values; + * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) + * 6. Generate a graph with gnuplot based on this data. + * 7. Determine the minimum number of unique values at which it is better to correct the error + * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. + * 7. Accordingly, update the constant in the function getThreshold() + * 8. Assemble ClickHouse. + */ struct UniqCombinedBiasData { using InterpolatedData = std::array; diff --git a/dbms/src/AggregateFunctions/UniquesHashSet.h b/dbms/src/AggregateFunctions/UniquesHashSet.h index b0036d4832d..bc9a65c1bb6 100644 --- a/dbms/src/AggregateFunctions/UniquesHashSet.h +++ b/dbms/src/AggregateFunctions/UniquesHashSet.h @@ -15,33 +15,33 @@ /** Approximate calculation of anything, as usual, is constructed according to the following scheme: -  * - some data structure is used to calculate the value of X; -  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); -  * - after processing all elements, the data structure is in some state S; -  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: -  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. -  */ + * - some data structure is used to calculate the value of X; + * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); + * - after processing all elements, the data structure is in some state S; + * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: + * at what real value X, the probability of finding the data structure in the obtained state S is maximal. + */ /** In particular, what is described below can be found by the name of the BJKST algorithm. -  */ + */ /** Very simple hash-set for approximate number of unique values. -  * Works like this: -  * - you can insert UInt64; -  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; -  * - the original value is not saved (lost); -  * - further all operations are made with these hashes; -  * - hash table is constructed according to the scheme: -  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); -  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); -  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; -  * - buffer growth by 2 times when filling more than 50%; -  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, -  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; -  * - if the situation repeats, then only elements dividing by 4, etc., are taken. -  * - the size() method returns an approximate number of elements that have been inserted into the set; -  * - there are methods for quick reading and writing in binary and text form. -  */ + * Works like this: + * - you can insert UInt64; + * - before insertion, first the hash function UInt64 -> UInt32 is calculated; + * - the original value is not saved (lost); + * - further all operations are made with these hashes; + * - hash table is constructed according to the scheme: + * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); + * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); + * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; + * - buffer growth by 2 times when filling more than 50%; + * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, + * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; + * - if the situation repeats, then only elements dividing by 4, etc., are taken. + * - the size() method returns an approximate number of elements that have been inserted into the set; + * - there are methods for quick reading and writing in binary and text form. + */ /// The maximum degree of buffer size before the values are discarded #define UNIQUES_HASH_MAX_SIZE_DEGREE 17 @@ -50,8 +50,8 @@ #define UNIQUES_HASH_MAX_SIZE (1ULL << (UNIQUES_HASH_MAX_SIZE_DEGREE - 1)) /** The number of least significant bits used for thinning. The remaining high-order bits are used to determine the position in the hash table. -  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) -  */ + * (high-order bits are taken because the younger bits will be constant after dropping some of the values) + */ #define UNIQUES_HASH_BITS_FOR_SKIP (32 - UNIQUES_HASH_MAX_SIZE_DEGREE) /// Initial buffer size degree @@ -59,8 +59,8 @@ /** This hash function is not the most optimal, but UniquesHashSet states counted with it, -  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. -  */ + * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. + */ struct UniquesHashSetDefaultHash { size_t operator() (UInt64 x) const diff --git a/dbms/src/Client/tests/test_connect.cpp b/dbms/src/Client/tests/test_connect.cpp index 75eb606cc97..50075cc24a6 100644 --- a/dbms/src/Client/tests/test_connect.cpp +++ b/dbms/src/Client/tests/test_connect.cpp @@ -9,9 +9,9 @@ /** In a loop it connects to the server and immediately breaks the connection. -  * Using the SO_LINGER option, we ensure that the connection is terminated by sending a RST packet (not FIN). -  * This behavior causes a bug in the TCPServer implementation in the Poco library. -  */ + * Using the SO_LINGER option, we ensure that the connection is terminated by sending a RST packet (not FIN). + * This behavior causes a bug in the TCPServer implementation in the Poco library. + */ int main(int argc, char ** argv) try { diff --git a/dbms/src/Common/InterruptListener.h b/dbms/src/Common/InterruptListener.h index eb818671037..37ec0533dd6 100644 --- a/dbms/src/Common/InterruptListener.h +++ b/dbms/src/Common/InterruptListener.h @@ -45,10 +45,10 @@ static int sigtimedwait(const sigset_t *set, siginfo_t *info, const struct times /** As long as there exists an object of this class - it blocks the INT signal, at the same time it lets you know if it came. -  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. -  * Use only one instance of this class at a time. -  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. -  */ + * This is necessary so that you can interrupt the execution of the request with Ctrl+C. + * Use only one instance of this class at a time. + * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. + */ class InterruptListener { private: From 4e413f4c2d693c657fe40907bded0bcf7e3c74ca Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Jan 2019 14:03:02 +0300 Subject: [PATCH 058/324] Move classes to separate files --- dbms/programs/performance-test/CMakeLists.txt | 8 +- dbms/programs/performance-test/JSONString.cpp | 63 +++ dbms/programs/performance-test/JSONString.h | 39 ++ .../performance-test/PerformanceTest.cpp | 452 +----------------- .../performance-test/StopConditionsSet.cpp | 63 +++ .../performance-test/StopConditionsSet.h | 40 ++ dbms/programs/performance-test/TestStats.cpp | 175 +++++++ dbms/programs/performance-test/TestStats.h | 83 ++++ .../performance-test/TestStopConditions.cpp | 26 + .../performance-test/TestStopConditions.h | 53 ++ 10 files changed, 562 insertions(+), 440 deletions(-) create mode 100644 dbms/programs/performance-test/JSONString.cpp create mode 100644 dbms/programs/performance-test/JSONString.h create mode 100644 dbms/programs/performance-test/StopConditionsSet.cpp create mode 100644 dbms/programs/performance-test/StopConditionsSet.h create mode 100644 dbms/programs/performance-test/TestStats.cpp create mode 100644 dbms/programs/performance-test/TestStats.h create mode 100644 dbms/programs/performance-test/TestStopConditions.cpp create mode 100644 dbms/programs/performance-test/TestStopConditions.h diff --git a/dbms/programs/performance-test/CMakeLists.txt b/dbms/programs/performance-test/CMakeLists.txt index f1a08172009..591a7180691 100644 --- a/dbms/programs/performance-test/CMakeLists.txt +++ b/dbms/programs/performance-test/CMakeLists.txt @@ -1,4 +1,10 @@ -add_library (clickhouse-performance-test-lib ${LINK_MODE} PerformanceTest.cpp) +add_library (clickhouse-performance-test-lib ${LINK_MODE} + JSONString.cpp + StopConditionsSet.cpp + TestStopConditions.cpp + TestStats.cpp + PerformanceTest.cpp +) target_link_libraries (clickhouse-performance-test-lib PRIVATE dbms clickhouse_common_io clickhouse_common_config ${Boost_PROGRAM_OPTIONS_LIBRARY}) target_include_directories (clickhouse-performance-test-lib SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) diff --git a/dbms/programs/performance-test/JSONString.cpp b/dbms/programs/performance-test/JSONString.cpp new file mode 100644 index 00000000000..abea80caf66 --- /dev/null +++ b/dbms/programs/performance-test/JSONString.cpp @@ -0,0 +1,63 @@ +#include "JSONString.h" + +#include +namespace DB +{ + +namespace +{ +String pad(size_t padding) +{ + return String(padding * 4, ' '); +} + +const std::regex NEW_LINE{"\n"}; +} + +void JSONString::set(const String key, String value, bool wrap) +{ + if (value.empty()) + value = "null"; + + bool reserved = (value[0] == '[' || value[0] == '{' || value == "null"); + if (!reserved && wrap) + value = '"' + std::regex_replace(value, NEW_LINE, "\\n") + '"'; + + content[key] = value; +} + +void JSONString::set(const String key, const std::vector & run_infos) +{ + String value = "[\n"; + + for (size_t i = 0; i < run_infos.size(); ++i) + { + value += pad(padding + 1) + run_infos[i].asString(padding + 2); + if (i != run_infos.size() - 1) + value += ','; + + value += "\n"; + } + + value += pad(padding) + ']'; + content[key] = value; +} + +String JSONString::asString(size_t cur_padding) const +{ + String repr = "{"; + + for (auto it = content.begin(); it != content.end(); ++it) + { + if (it != content.begin()) + repr += ','; + /// construct "key": "value" string with padding + repr += "\n" + pad(cur_padding) + '"' + it->first + '"' + ": " + it->second; + } + + repr += "\n" + pad(cur_padding - 1) + '}'; + return repr; +} + + +} diff --git a/dbms/programs/performance-test/JSONString.h b/dbms/programs/performance-test/JSONString.h new file mode 100644 index 00000000000..ee83be5e9a6 --- /dev/null +++ b/dbms/programs/performance-test/JSONString.h @@ -0,0 +1,39 @@ +#pragma once +#include + +#include +#include +#include +#include + +namespace DB +{ + +/// NOTE The code is totally wrong. +class JSONString +{ +private: + std::map content; + size_t padding; + +public: + explicit JSONString(size_t padding_ = 1) : padding(padding_) {} + + void set(const String key, String value, bool wrap = true); + + template + std::enable_if_t> set(const String key, T value) + { + set(key, std::to_string(value), /*wrap= */ false); + } + + void set(const String key, const std::vector & run_infos); + + String asString() const + { + return asString(padding); + } + + String asString(size_t cur_padding) const; +}; +} diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index e91365aeade..d5bfcc85c60 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -7,6 +7,7 @@ #include #include #include + #include #include #include @@ -34,6 +35,11 @@ #include #include +#include "JSONString.h" +#include "StopConditionsSet.h" +#include "TestStopConditions.h" +#include "TestStats.h" + #ifndef __clang__ #pragma GCC optimize("-fno-var-tracking-assignments") #endif @@ -45,9 +51,7 @@ */ namespace fs = boost::filesystem; using String = std::string; -const String FOUR_SPACES = " "; const std::regex QUOTE_REGEX{"\""}; -const std::regex NEW_LINE{"\n"}; namespace DB { @@ -59,439 +63,9 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; } -static String pad(size_t padding) -{ - return String(padding * 4, ' '); -} - - -/// NOTE The code is totally wrong. -class JSONString -{ -private: - std::map content; - size_t padding; - -public: - explicit JSONString(size_t padding_ = 1) : padding(padding_) {} - - void set(const String key, String value, bool wrap = true) - { - if (value.empty()) - value = "null"; - - bool reserved = (value[0] == '[' || value[0] == '{' || value == "null"); - if (!reserved && wrap) - value = '"' + std::regex_replace(value, NEW_LINE, "\\n") + '"'; - - content[key] = value; - } - - template - std::enable_if_t> set(const String key, T value) - { - set(key, std::to_string(value), /*wrap= */ false); - } - - void set(const String key, const std::vector & run_infos) - { - String value = "[\n"; - - for (size_t i = 0; i < run_infos.size(); ++i) - { - value += pad(padding + 1) + run_infos[i].asString(padding + 2); - if (i != run_infos.size() - 1) - value += ','; - - value += "\n"; - } - - value += pad(padding) + ']'; - content[key] = value; - } - - String asString() const - { - return asString(padding); - } - - String asString(size_t cur_padding) const - { - String repr = "{"; - - for (auto it = content.begin(); it != content.end(); ++it) - { - if (it != content.begin()) - repr += ','; - /// construct "key": "value" string with padding - repr += "\n" + pad(cur_padding) + '"' + it->first + '"' + ": " + it->second; - } - - repr += "\n" + pad(cur_padding - 1) + '}'; - return repr; - } -}; - using ConfigurationPtr = Poco::AutoPtr; -/// A set of supported stop conditions. -struct StopConditionsSet -{ - void loadFromConfig(const ConfigurationPtr & stop_conditions_view) - { - using Keys = std::vector; - Keys keys; - stop_conditions_view->keys(keys); - - for (const String & key : keys) - { - if (key == "total_time_ms") - total_time_ms.value = stop_conditions_view->getUInt64(key); - else if (key == "rows_read") - rows_read.value = stop_conditions_view->getUInt64(key); - else if (key == "bytes_read_uncompressed") - bytes_read_uncompressed.value = stop_conditions_view->getUInt64(key); - else if (key == "iterations") - iterations.value = stop_conditions_view->getUInt64(key); - else if (key == "min_time_not_changing_for_ms") - min_time_not_changing_for_ms.value = stop_conditions_view->getUInt64(key); - else if (key == "max_speed_not_changing_for_ms") - max_speed_not_changing_for_ms.value = stop_conditions_view->getUInt64(key); - else if (key == "average_speed_not_changing_for_ms") - average_speed_not_changing_for_ms.value = stop_conditions_view->getUInt64(key); - else - throw DB::Exception("Met unkown stop condition: " + key, DB::ErrorCodes::LOGICAL_ERROR); - - ++initialized_count; - } - } - - void reset() - { - total_time_ms.fulfilled = false; - rows_read.fulfilled = false; - bytes_read_uncompressed.fulfilled = false; - iterations.fulfilled = false; - min_time_not_changing_for_ms.fulfilled = false; - max_speed_not_changing_for_ms.fulfilled = false; - average_speed_not_changing_for_ms.fulfilled = false; - - fulfilled_count = 0; - } - - /// Note: only conditions with UInt64 minimal thresholds are supported. - /// I.e. condition is fulfilled when value is exceeded. - struct StopCondition - { - UInt64 value = 0; - bool fulfilled = false; - }; - - void report(UInt64 value, StopCondition & condition) - { - if (condition.value && !condition.fulfilled && value >= condition.value) - { - condition.fulfilled = true; - ++fulfilled_count; - } - } - - StopCondition total_time_ms; - StopCondition rows_read; - StopCondition bytes_read_uncompressed; - StopCondition iterations; - StopCondition min_time_not_changing_for_ms; - StopCondition max_speed_not_changing_for_ms; - StopCondition average_speed_not_changing_for_ms; - - size_t initialized_count = 0; - size_t fulfilled_count = 0; -}; - -/// Stop conditions for a test run. The running test will be terminated in either of two conditions: -/// 1. All conditions marked 'all_of' are fulfilled -/// or -/// 2. Any condition marked 'any_of' is fulfilled -class TestStopConditions -{ -public: - void loadFromConfig(ConfigurationPtr & stop_conditions_config) - { - if (stop_conditions_config->has("all_of")) - { - ConfigurationPtr config_all_of(stop_conditions_config->createView("all_of")); - conditions_all_of.loadFromConfig(config_all_of); - } - if (stop_conditions_config->has("any_of")) - { - ConfigurationPtr config_any_of(stop_conditions_config->createView("any_of")); - conditions_any_of.loadFromConfig(config_any_of); - } - } - - bool empty() const - { - return !conditions_all_of.initialized_count && !conditions_any_of.initialized_count; - } - -#define DEFINE_REPORT_FUNC(FUNC_NAME, CONDITION) \ - void FUNC_NAME(UInt64 value) \ - { \ - conditions_all_of.report(value, conditions_all_of.CONDITION); \ - conditions_any_of.report(value, conditions_any_of.CONDITION); \ - } - - DEFINE_REPORT_FUNC(reportTotalTime, total_time_ms) - DEFINE_REPORT_FUNC(reportRowsRead, rows_read) - DEFINE_REPORT_FUNC(reportBytesReadUncompressed, bytes_read_uncompressed) - DEFINE_REPORT_FUNC(reportIterations, iterations) - DEFINE_REPORT_FUNC(reportMinTimeNotChangingFor, min_time_not_changing_for_ms) - DEFINE_REPORT_FUNC(reportMaxSpeedNotChangingFor, max_speed_not_changing_for_ms) - DEFINE_REPORT_FUNC(reportAverageSpeedNotChangingFor, average_speed_not_changing_for_ms) - -#undef REPORT - - bool areFulfilled() const - { - return (conditions_all_of.initialized_count && conditions_all_of.fulfilled_count >= conditions_all_of.initialized_count) - || (conditions_any_of.initialized_count && conditions_any_of.fulfilled_count); - } - - void reset() - { - conditions_all_of.reset(); - conditions_any_of.reset(); - } - -private: - StopConditionsSet conditions_all_of; - StopConditionsSet conditions_any_of; -}; - -struct Stats -{ - Stopwatch watch; - Stopwatch watch_per_query; - Stopwatch min_time_watch; - Stopwatch max_rows_speed_watch; - Stopwatch max_bytes_speed_watch; - Stopwatch avg_rows_speed_watch; - Stopwatch avg_bytes_speed_watch; - - bool last_query_was_cancelled = false; - - size_t queries = 0; - - size_t total_rows_read = 0; - size_t total_bytes_read = 0; - - size_t last_query_rows_read = 0; - size_t last_query_bytes_read = 0; - - using Sampler = ReservoirSampler; - Sampler sampler{1 << 16}; - - /// min_time in ms - UInt64 min_time = std::numeric_limits::max(); - double total_time = 0; - - double max_rows_speed = 0; - double max_bytes_speed = 0; - - double avg_rows_speed_value = 0; - double avg_rows_speed_first = 0; - static double avg_rows_speed_precision; - - double avg_bytes_speed_value = 0; - double avg_bytes_speed_first = 0; - static double avg_bytes_speed_precision; - - size_t number_of_rows_speed_info_batches = 0; - size_t number_of_bytes_speed_info_batches = 0; - - bool ready = false; // check if a query wasn't interrupted by SIGINT - String exception; - - String getStatisticByName(const String & statistic_name) - { - if (statistic_name == "min_time") - { - return std::to_string(min_time) + "ms"; - } - if (statistic_name == "quantiles") - { - String result = "\n"; - - for (double percent = 10; percent <= 90; percent += 10) - { - result += FOUR_SPACES + std::to_string((percent / 100)); - result += ": " + std::to_string(sampler.quantileInterpolated(percent / 100.0)); - result += "\n"; - } - result += FOUR_SPACES + "0.95: " + std::to_string(sampler.quantileInterpolated(95 / 100.0)) + "\n"; - result += FOUR_SPACES + "0.99: " + std::to_string(sampler.quantileInterpolated(99 / 100.0)) + "\n"; - result += FOUR_SPACES + "0.999: " + std::to_string(sampler.quantileInterpolated(99.9 / 100.)) + "\n"; - result += FOUR_SPACES + "0.9999: " + std::to_string(sampler.quantileInterpolated(99.99 / 100.)); - - return result; - } - if (statistic_name == "total_time") - { - return std::to_string(total_time) + "s"; - } - if (statistic_name == "queries_per_second") - { - return std::to_string(queries / total_time); - } - if (statistic_name == "rows_per_second") - { - return std::to_string(total_rows_read / total_time); - } - if (statistic_name == "bytes_per_second") - { - return std::to_string(total_bytes_read / total_time); - } - - if (statistic_name == "max_rows_per_second") - { - return std::to_string(max_rows_speed); - } - if (statistic_name == "max_bytes_per_second") - { - return std::to_string(max_bytes_speed); - } - if (statistic_name == "avg_rows_per_second") - { - return std::to_string(avg_rows_speed_value); - } - if (statistic_name == "avg_bytes_per_second") - { - return std::to_string(avg_bytes_speed_value); - } - - return ""; - } - - void update_min_time(const UInt64 min_time_candidate) - { - if (min_time_candidate < min_time) - { - min_time = min_time_candidate; - min_time_watch.restart(); - } - } - - void update_average_speed(const double new_speed_info, - Stopwatch & avg_speed_watch, - size_t & number_of_info_batches, - double precision, - double & avg_speed_first, - double & avg_speed_value) - { - avg_speed_value = ((avg_speed_value * number_of_info_batches) + new_speed_info); - ++number_of_info_batches; - avg_speed_value /= number_of_info_batches; - - if (avg_speed_first == 0) - { - avg_speed_first = avg_speed_value; - } - - if (std::abs(avg_speed_value - avg_speed_first) >= precision) - { - avg_speed_first = avg_speed_value; - avg_speed_watch.restart(); - } - } - - void update_max_speed(const size_t max_speed_candidate, Stopwatch & max_speed_watch, double & max_speed) - { - if (max_speed_candidate > max_speed) - { - max_speed = max_speed_candidate; - max_speed_watch.restart(); - } - } - - void add(size_t rows_read_inc, size_t bytes_read_inc) - { - total_rows_read += rows_read_inc; - total_bytes_read += bytes_read_inc; - last_query_rows_read += rows_read_inc; - last_query_bytes_read += bytes_read_inc; - - double new_rows_speed = last_query_rows_read / watch_per_query.elapsedSeconds(); - double new_bytes_speed = last_query_bytes_read / watch_per_query.elapsedSeconds(); - - /// Update rows speed - update_max_speed(new_rows_speed, max_rows_speed_watch, max_rows_speed); - update_average_speed(new_rows_speed, - avg_rows_speed_watch, - number_of_rows_speed_info_batches, - avg_rows_speed_precision, - avg_rows_speed_first, - avg_rows_speed_value); - /// Update bytes speed - update_max_speed(new_bytes_speed, max_bytes_speed_watch, max_bytes_speed); - update_average_speed(new_bytes_speed, - avg_bytes_speed_watch, - number_of_bytes_speed_info_batches, - avg_bytes_speed_precision, - avg_bytes_speed_first, - avg_bytes_speed_value); - } - - void updateQueryInfo() - { - ++queries; - sampler.insert(watch_per_query.elapsedSeconds()); - update_min_time(watch_per_query.elapsed() / (1000 * 1000)); /// ns to ms - } - - void setTotalTime() - { - total_time = watch.elapsedSeconds(); - } - - void clear() - { - watch.restart(); - watch_per_query.restart(); - min_time_watch.restart(); - max_rows_speed_watch.restart(); - max_bytes_speed_watch.restart(); - avg_rows_speed_watch.restart(); - avg_bytes_speed_watch.restart(); - - last_query_was_cancelled = false; - - sampler.clear(); - - queries = 0; - total_rows_read = 0; - total_bytes_read = 0; - last_query_rows_read = 0; - last_query_bytes_read = 0; - - min_time = std::numeric_limits::max(); - total_time = 0; - max_rows_speed = 0; - max_bytes_speed = 0; - avg_rows_speed_value = 0; - avg_bytes_speed_value = 0; - avg_rows_speed_first = 0; - avg_bytes_speed_first = 0; - avg_rows_speed_precision = 0.001; - avg_bytes_speed_precision = 0.001; - number_of_rows_speed_info_batches = 0; - number_of_bytes_speed_info_batches = 0; - } -}; - -double Stats::avg_rows_speed_precision = 0.001; -double Stats::avg_bytes_speed_precision = 0.001; - class PerformanceTest : public Poco::Util::Application { public: @@ -618,7 +192,7 @@ private: }; size_t times_to_run = 1; - std::vector statistics_by_run; + std::vector statistics_by_run; /// Removes configurations that has a given value. If leave is true, the logic is reversed. void removeConfigurationsIf( @@ -876,12 +450,12 @@ private: if (std::find(config_settings.begin(), config_settings.end(), "average_rows_speed_precision") != config_settings.end()) { - Stats::avg_rows_speed_precision = test_config->getDouble("settings.average_rows_speed_precision"); + TestStats::avg_rows_speed_precision = test_config->getDouble("settings.average_rows_speed_precision"); } if (std::find(config_settings.begin(), config_settings.end(), "average_bytes_speed_precision") != config_settings.end()) { - Stats::avg_bytes_speed_precision = test_config->getDouble("settings.average_bytes_speed_precision"); + TestStats::avg_bytes_speed_precision = test_config->getDouble("settings.average_bytes_speed_precision"); } } @@ -1062,7 +636,7 @@ private: for (const auto & [query, run_index] : queries_with_indexes) { TestStopConditions & stop_conditions = stop_conditions_by_run[run_index]; - Stats & statistics = statistics_by_run[run_index]; + TestStats & statistics = statistics_by_run[run_index]; statistics.clear(); try @@ -1093,7 +667,7 @@ private: } } - void execute(const Query & query, Stats & statistics, TestStopConditions & stop_conditions) + void execute(const Query & query, TestStats & statistics, TestStopConditions & stop_conditions) { statistics.watch_per_query.restart(); statistics.last_query_was_cancelled = false; @@ -1117,7 +691,7 @@ private: } void checkFulfilledConditionsAndUpdate( - const Progress & progress, RemoteBlockInputStream & stream, Stats & statistics, TestStopConditions & stop_conditions) + const Progress & progress, RemoteBlockInputStream & stream, TestStats & statistics, TestStopConditions & stop_conditions) { statistics.add(progress.rows, progress.bytes); @@ -1256,7 +830,7 @@ public: { for (size_t number_of_launch = 0; number_of_launch < times_to_run; ++number_of_launch) { - Stats & statistics = statistics_by_run[number_of_launch * queries.size() + query_index]; + TestStats & statistics = statistics_by_run[number_of_launch * queries.size() + query_index]; if (!statistics.ready) continue; diff --git a/dbms/programs/performance-test/StopConditionsSet.cpp b/dbms/programs/performance-test/StopConditionsSet.cpp new file mode 100644 index 00000000000..624c5b48a29 --- /dev/null +++ b/dbms/programs/performance-test/StopConditionsSet.cpp @@ -0,0 +1,63 @@ +#include "StopConditionsSet.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + +void StopConditionsSet::loadFromConfig(const ConfigurationPtr & stop_conditions_view) +{ + std::vector keys; + stop_conditions_view->keys(keys); + + for (const String & key : keys) + { + if (key == "total_time_ms") + total_time_ms.value = stop_conditions_view->getUInt64(key); + else if (key == "rows_read") + rows_read.value = stop_conditions_view->getUInt64(key); + else if (key == "bytes_read_uncompressed") + bytes_read_uncompressed.value = stop_conditions_view->getUInt64(key); + else if (key == "iterations") + iterations.value = stop_conditions_view->getUInt64(key); + else if (key == "min_time_not_changing_for_ms") + min_time_not_changing_for_ms.value = stop_conditions_view->getUInt64(key); + else if (key == "max_speed_not_changing_for_ms") + max_speed_not_changing_for_ms.value = stop_conditions_view->getUInt64(key); + else if (key == "average_speed_not_changing_for_ms") + average_speed_not_changing_for_ms.value = stop_conditions_view->getUInt64(key); + else + throw DB::Exception("Met unkown stop condition: " + key, DB::ErrorCodes::LOGICAL_ERROR); + } + ++initialized_count; +} + +void StopConditionsSet::reset() +{ + total_time_ms.fulfilled = false; + rows_read.fulfilled = false; + bytes_read_uncompressed.fulfilled = false; + iterations.fulfilled = false; + min_time_not_changing_for_ms.fulfilled = false; + max_speed_not_changing_for_ms.fulfilled = false; + average_speed_not_changing_for_ms.fulfilled = false; + + fulfilled_count = 0; +} + +void StopConditionsSet::report(UInt64 value, StopConditionsSet::StopCondition & condition) +{ + if (condition.value && !condition.fulfilled && value >= condition.value) + { + condition.fulfilled = true; + ++fulfilled_count; + } +} + + + +} diff --git a/dbms/programs/performance-test/StopConditionsSet.h b/dbms/programs/performance-test/StopConditionsSet.h new file mode 100644 index 00000000000..e83a4251bd0 --- /dev/null +++ b/dbms/programs/performance-test/StopConditionsSet.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +using ConfigurationPtr = Poco::AutoPtr; + +/// A set of supported stop conditions. +struct StopConditionsSet +{ + void loadFromConfig(const ConfigurationPtr & stop_conditions_view); + void reset(); + + /// Note: only conditions with UInt64 minimal thresholds are supported. + /// I.e. condition is fulfilled when value is exceeded. + struct StopCondition + { + UInt64 value = 0; + bool fulfilled = false; + }; + + void report(UInt64 value, StopCondition & condition); + + StopCondition total_time_ms; + StopCondition rows_read; + StopCondition bytes_read_uncompressed; + StopCondition iterations; + StopCondition min_time_not_changing_for_ms; + StopCondition max_speed_not_changing_for_ms; + StopCondition average_speed_not_changing_for_ms; + + size_t initialized_count = 0; + size_t fulfilled_count = 0; +}; + +} diff --git a/dbms/programs/performance-test/TestStats.cpp b/dbms/programs/performance-test/TestStats.cpp new file mode 100644 index 00000000000..163aefdc98d --- /dev/null +++ b/dbms/programs/performance-test/TestStats.cpp @@ -0,0 +1,175 @@ +#include "TestStats.h" +namespace DB +{ + +namespace +{ +const String FOUR_SPACES = " "; +} + +String TestStats::getStatisticByName(const String & statistic_name) +{ + if (statistic_name == "min_time") + return std::to_string(min_time) + "ms"; + + if (statistic_name == "quantiles") + { + String result = "\n"; + + for (double percent = 10; percent <= 90; percent += 10) + { + result += FOUR_SPACES + std::to_string((percent / 100)); + result += ": " + std::to_string(sampler.quantileInterpolated(percent / 100.0)); + result += "\n"; + } + result += FOUR_SPACES + "0.95: " + std::to_string(sampler.quantileInterpolated(95 / 100.0)) + "\n"; + result += FOUR_SPACES + "0.99: " + std::to_string(sampler.quantileInterpolated(99 / 100.0)) + "\n"; + result += FOUR_SPACES + "0.999: " + std::to_string(sampler.quantileInterpolated(99.9 / 100.)) + "\n"; + result += FOUR_SPACES + "0.9999: " + std::to_string(sampler.quantileInterpolated(99.99 / 100.)); + + return result; + } + if (statistic_name == "total_time") + return std::to_string(total_time) + "s"; + + if (statistic_name == "queries_per_second") + return std::to_string(queries / total_time); + + if (statistic_name == "rows_per_second") + return std::to_string(total_rows_read / total_time); + + if (statistic_name == "bytes_per_second") + return std::to_string(total_bytes_read / total_time); + + if (statistic_name == "max_rows_per_second") + return std::to_string(max_rows_speed); + + if (statistic_name == "max_bytes_per_second") + return std::to_string(max_bytes_speed); + + if (statistic_name == "avg_rows_per_second") + return std::to_string(avg_rows_speed_value); + + if (statistic_name == "avg_bytes_per_second") + return std::to_string(avg_bytes_speed_value); + + return ""; +} + + +void TestStats::update_min_time(UInt64 min_time_candidate) +{ + if (min_time_candidate < min_time) + { + min_time = min_time_candidate; + min_time_watch.restart(); + } +} + +void TestStats::update_max_speed( + size_t max_speed_candidate, + Stopwatch & max_speed_watch, + double & max_speed) +{ + if (max_speed_candidate > max_speed) + { + max_speed = max_speed_candidate; + max_speed_watch.restart(); + } +} + + +void TestStats::update_average_speed( + double new_speed_info, + Stopwatch & avg_speed_watch, + size_t & number_of_info_batches, + double precision, + double & avg_speed_first, + double & avg_speed_value) +{ + avg_speed_value = ((avg_speed_value * number_of_info_batches) + new_speed_info); + ++number_of_info_batches; + avg_speed_value /= number_of_info_batches; + + if (avg_speed_first == 0) + { + avg_speed_first = avg_speed_value; + } + + if (std::abs(avg_speed_value - avg_speed_first) >= precision) + { + avg_speed_first = avg_speed_value; + avg_speed_watch.restart(); + } +} + +void TestStats::add(size_t rows_read_inc, size_t bytes_read_inc) +{ + total_rows_read += rows_read_inc; + total_bytes_read += bytes_read_inc; + last_query_rows_read += rows_read_inc; + last_query_bytes_read += bytes_read_inc; + + double new_rows_speed = last_query_rows_read / watch_per_query.elapsedSeconds(); + double new_bytes_speed = last_query_bytes_read / watch_per_query.elapsedSeconds(); + + /// Update rows speed + update_max_speed(new_rows_speed, max_rows_speed_watch, max_rows_speed); + update_average_speed(new_rows_speed, + avg_rows_speed_watch, + number_of_rows_speed_info_batches, + avg_rows_speed_precision, + avg_rows_speed_first, + avg_rows_speed_value); + /// Update bytes speed + update_max_speed(new_bytes_speed, max_bytes_speed_watch, max_bytes_speed); + update_average_speed(new_bytes_speed, + avg_bytes_speed_watch, + number_of_bytes_speed_info_batches, + avg_bytes_speed_precision, + avg_bytes_speed_first, + avg_bytes_speed_value); +} + +void TestStats::updateQueryInfo() +{ + ++queries; + sampler.insert(watch_per_query.elapsedSeconds()); + update_min_time(watch_per_query.elapsed() / (1000 * 1000)); /// ns to ms +} + +void TestStats::clear() +{ + watch.restart(); + watch_per_query.restart(); + min_time_watch.restart(); + max_rows_speed_watch.restart(); + max_bytes_speed_watch.restart(); + avg_rows_speed_watch.restart(); + avg_bytes_speed_watch.restart(); + + last_query_was_cancelled = false; + + sampler.clear(); + + queries = 0; + total_rows_read = 0; + total_bytes_read = 0; + last_query_rows_read = 0; + last_query_bytes_read = 0; + + min_time = std::numeric_limits::max(); + total_time = 0; + max_rows_speed = 0; + max_bytes_speed = 0; + avg_rows_speed_value = 0; + avg_bytes_speed_value = 0; + avg_rows_speed_first = 0; + avg_bytes_speed_first = 0; + avg_rows_speed_precision = 0.001; + avg_bytes_speed_precision = 0.001; + number_of_rows_speed_info_batches = 0; + number_of_bytes_speed_info_batches = 0; +} + +} diff --git a/dbms/programs/performance-test/TestStats.h b/dbms/programs/performance-test/TestStats.h new file mode 100644 index 00000000000..41a8efc3beb --- /dev/null +++ b/dbms/programs/performance-test/TestStats.h @@ -0,0 +1,83 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +struct TestStats +{ + Stopwatch watch; + Stopwatch watch_per_query; + Stopwatch min_time_watch; + Stopwatch max_rows_speed_watch; + Stopwatch max_bytes_speed_watch; + Stopwatch avg_rows_speed_watch; + Stopwatch avg_bytes_speed_watch; + + bool last_query_was_cancelled = false; + + size_t queries = 0; + + size_t total_rows_read = 0; + size_t total_bytes_read = 0; + + size_t last_query_rows_read = 0; + size_t last_query_bytes_read = 0; + + using Sampler = ReservoirSampler; + Sampler sampler{1 << 16}; + + /// min_time in ms + UInt64 min_time = std::numeric_limits::max(); + double total_time = 0; + + double max_rows_speed = 0; + double max_bytes_speed = 0; + + double avg_rows_speed_value = 0; + double avg_rows_speed_first = 0; + static inline double avg_rows_speed_precision = 0.001; + + double avg_bytes_speed_value = 0; + double avg_bytes_speed_first = 0; + static inline double avg_bytes_speed_precision = 0.001; + + size_t number_of_rows_speed_info_batches = 0; + size_t number_of_bytes_speed_info_batches = 0; + + bool ready = false; // check if a query wasn't interrupted by SIGINT + String exception; + + String getStatisticByName(const String & statistic_name); + + void update_min_time(UInt64 min_time_candidate); + + void update_average_speed( + double new_speed_info, + Stopwatch & avg_speed_watch, + size_t & number_of_info_batches, + double precision, + double & avg_speed_first, + double & avg_speed_value); + + void update_max_speed( + size_t max_speed_candidate, + Stopwatch & max_speed_watch, + double & max_speed); + + void add(size_t rows_read_inc, size_t bytes_read_inc); + + void updateQueryInfo(); + + void setTotalTime() + { + total_time = watch.elapsedSeconds(); + } + + void clear(); +}; + +} diff --git a/dbms/programs/performance-test/TestStopConditions.cpp b/dbms/programs/performance-test/TestStopConditions.cpp new file mode 100644 index 00000000000..bc608e4001a --- /dev/null +++ b/dbms/programs/performance-test/TestStopConditions.cpp @@ -0,0 +1,26 @@ +#include "TestStopConditions.h" + +namespace DB +{ + +void TestStopConditions::loadFromConfig(ConfigurationPtr & stop_conditions_config) +{ + if (stop_conditions_config->has("all_of")) + { + ConfigurationPtr config_all_of(stop_conditions_config->createView("all_of")); + conditions_all_of.loadFromConfig(config_all_of); + } + if (stop_conditions_config->has("any_of")) + { + ConfigurationPtr config_any_of(stop_conditions_config->createView("any_of")); + conditions_any_of.loadFromConfig(config_any_of); + } +} + +bool TestStopConditions::areFulfilled() const +{ + return (conditions_all_of.initialized_count && conditions_all_of.fulfilled_count >= conditions_all_of.initialized_count) + || (conditions_any_of.initialized_count && conditions_any_of.fulfilled_count); +} + +} diff --git a/dbms/programs/performance-test/TestStopConditions.h b/dbms/programs/performance-test/TestStopConditions.h new file mode 100644 index 00000000000..91f1baa1ced --- /dev/null +++ b/dbms/programs/performance-test/TestStopConditions.h @@ -0,0 +1,53 @@ +#pragma once +#include "StopConditionsSet.h" +#include + +namespace DB +{ +/// Stop conditions for a test run. The running test will be terminated in either of two conditions: +/// 1. All conditions marked 'all_of' are fulfilled +/// or +/// 2. Any condition marked 'any_of' is fulfilled + +using ConfigurationPtr = Poco::AutoPtr; + +class TestStopConditions +{ +public: + void loadFromConfig(ConfigurationPtr & stop_conditions_config); + inline bool empty() const + { + return !conditions_all_of.initialized_count && !conditions_any_of.initialized_count; + } + +#define DEFINE_REPORT_FUNC(FUNC_NAME, CONDITION) \ + void FUNC_NAME(UInt64 value) \ + { \ + conditions_all_of.report(value, conditions_all_of.CONDITION); \ + conditions_any_of.report(value, conditions_any_of.CONDITION); \ + } + + DEFINE_REPORT_FUNC(reportTotalTime, total_time_ms) + DEFINE_REPORT_FUNC(reportRowsRead, rows_read) + DEFINE_REPORT_FUNC(reportBytesReadUncompressed, bytes_read_uncompressed) + DEFINE_REPORT_FUNC(reportIterations, iterations) + DEFINE_REPORT_FUNC(reportMinTimeNotChangingFor, min_time_not_changing_for_ms) + DEFINE_REPORT_FUNC(reportMaxSpeedNotChangingFor, max_speed_not_changing_for_ms) + DEFINE_REPORT_FUNC(reportAverageSpeedNotChangingFor, average_speed_not_changing_for_ms) + +#undef REPORT + + bool areFulfilled() const; + + void reset() + { + conditions_all_of.reset(); + conditions_any_of.reset(); + } + +private: + StopConditionsSet conditions_all_of; + StopConditionsSet conditions_any_of; +}; + +} From a0fafefc96bc6b569f0f55d0f97ed20c7e65bb85 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 25 Jan 2019 14:43:19 +0300 Subject: [PATCH 059/324] minor change - add Aliases.h --- dbms/src/Interpreters/Aliases.h | 15 +++++++++++++++ dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h | 3 +-- .../OptimizeIfWithConstantConditionVisitor.h | 6 +----- dbms/src/Interpreters/QueryAliasesVisitor.h | 4 +--- dbms/src/Interpreters/QueryNormalizer.h | 4 ++-- dbms/src/Interpreters/SyntaxAnalyzer.h | 2 +- 6 files changed, 21 insertions(+), 13 deletions(-) create mode 100644 dbms/src/Interpreters/Aliases.h diff --git a/dbms/src/Interpreters/Aliases.h b/dbms/src/Interpreters/Aliases.h new file mode 100644 index 00000000000..80976c7551f --- /dev/null +++ b/dbms/src/Interpreters/Aliases.h @@ -0,0 +1,15 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +class IAST; +using ASTPtr = std::shared_ptr; + +using Aliases = std::unordered_map; + +} diff --git a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h index d345ee42dbe..0c19ffe7387 100644 --- a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -10,6 +10,7 @@ #include #include +#include namespace DB @@ -28,8 +29,6 @@ class ArrayJoinedColumnsMatcher public: struct Data { - using Aliases = std::unordered_map; - const Aliases & aliases; NameToNameMap & array_join_name_to_alias; NameToNameMap & array_join_alias_to_name; diff --git a/dbms/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h b/dbms/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h index ee738ec05e2..05d0330196b 100644 --- a/dbms/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h +++ b/dbms/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h @@ -1,8 +1,6 @@ #pragma once -#include - -#include +#include namespace DB { @@ -12,8 +10,6 @@ namespace DB class OptimizeIfWithConstantConditionVisitor { public: - using Aliases = std::unordered_map; - OptimizeIfWithConstantConditionVisitor(Aliases & aliases_) : aliases(aliases_) {} diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.h b/dbms/src/Interpreters/QueryAliasesVisitor.h index de6080f2609..5ca712a1007 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.h +++ b/dbms/src/Interpreters/QueryAliasesVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -11,8 +11,6 @@ class ASTSubquery; struct ASTTableExpression; struct ASTArrayJoin; -using Aliases = std::unordered_map; - /// Visits AST node to collect aliases. class QueryAliasesMatcher { diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 671ddf1b106..62aaa09bb34 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -1,11 +1,12 @@ #pragma once #include +#include #include #include #include -#include +#include namespace DB { @@ -45,7 +46,6 @@ class QueryNormalizer }; public: - using Aliases = std::unordered_map; using TableWithColumnNames = std::pair; struct Data diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index d34b16c1a40..4450881ee68 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -16,7 +17,6 @@ struct SyntaxAnalyzerResult NamesAndTypesList source_columns; - using Aliases = std::unordered_map; Aliases aliases; /// Which column is needed to be ARRAY-JOIN'ed to get the specified. From 4bb487763aeaf59e7a9c9ec4caf870d60f32fa73 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Jan 2019 15:45:35 +0300 Subject: [PATCH 060/324] Make it work #4126 --- cmake/find_zstd.cmake | 1 + contrib/librdkafka-cmake/CMakeLists.txt | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/cmake/find_zstd.cmake b/cmake/find_zstd.cmake index 0e6db94a7c0..24bc851ed57 100644 --- a/cmake/find_zstd.cmake +++ b/cmake/find_zstd.cmake @@ -14,6 +14,7 @@ if (ZSTD_LIBRARY AND ZSTD_INCLUDE_DIR) else () set (USE_INTERNAL_ZSTD_LIBRARY 1) set (ZSTD_LIBRARY zstd) + set (ZSTD_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/zstd/lib) endif () message (STATUS "Using zstd: ${ZSTD_INCLUDE_DIR} : ${ZSTD_LIBRARY}") diff --git a/contrib/librdkafka-cmake/CMakeLists.txt b/contrib/librdkafka-cmake/CMakeLists.txt index fb27895c3da..62337f60fb5 100644 --- a/contrib/librdkafka-cmake/CMakeLists.txt +++ b/contrib/librdkafka-cmake/CMakeLists.txt @@ -60,5 +60,6 @@ set(SRCS add_library(rdkafka ${LINK_MODE} ${SRCS}) target_include_directories(rdkafka SYSTEM PUBLIC include) -target_include_directories(rdkafka SYSTEM PUBLIC ${RDKAFKA_SOURCE_DIR}) -target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) +target_include_directories(rdkafka SYSTEM PUBLIC ${RDKAFKA_SOURCE_DIR}) # Because weird logic with "include_next" is used. +target_include_directories(rdkafka SYSTEM PRIVATE ${ZSTD_INCLUDE_DIR}/common) # Because wrong path to "zstd_errors.h" is used. +target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${ZSTD_LIBRARY} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) From 02f236e45709fca2e8d82e5caf644b1c27487ff0 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 25 Jan 2019 16:15:29 +0300 Subject: [PATCH 061/324] one more test --- .../0_stateless/00819_ast_refactoring_bugs.reference | 1 + .../queries/0_stateless/00819_ast_refactoring_bugs.sql | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference index e69de29bb2d..26c4a78a1a9 100644 --- a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference +++ b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference @@ -0,0 +1 @@ +1 [1] diff --git a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql index 27171f2e0de..010d03920c7 100644 --- a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql +++ b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql @@ -20,3 +20,11 @@ ARRAY JOIN WHERE CounterID = 100500; DROP TABLE test.visits; + +select u, cumSum from ( + select u, min(d) mn, max(d) mx, groupArray(d) dg, groupArray(v) vg, + arrayMap(x -> x + mn, range(toUInt32(mx - mn + 1))) days, + toString(arrayCumSum(arrayMap( x -> vg[indexOf(dg, x)] , days))) cumSum + from (select 1 u, today()-1 d, 1 v) + group by u +); From d9195cda99fae43857a0e5bde0958b044dbc9d02 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 14:06:21 +0100 Subject: [PATCH 062/324] implement `getWidenType()` on IDataType This is essentially to help implementing the `getReturnType` of aggregation function susceptible to provoque overflow (such as `sumMap`). --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/DataTypes/DataTypesDecimal.cpp | 8 ++++++++ dbms/src/DataTypes/DataTypesDecimal.h | 2 ++ dbms/src/DataTypes/DataTypesNumber.h | 7 +++++++ dbms/src/DataTypes/IDataType.cpp | 5 +++++ dbms/src/DataTypes/IDataType.h | 9 +++++++++ 6 files changed, 32 insertions(+) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index f06a88c96da..d8749c1b1e7 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -412,6 +412,7 @@ namespace ErrorCodes extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD = 435; extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE = 436; extern const int PROTOBUF_FIELD_NOT_REPEATED = 437; + extern const int DATA_TYPE_CANNOT_BE_WIDEN = 438; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index 439a98928ea..f8f3084b254 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -148,6 +148,14 @@ Field DataTypeDecimal::getDefault() const } +template +DataTypePtr DataTypeDecimal::getWidenDataType() const +{ + using WidenDataType = DataTypeDecimal; + return std::make_shared(WidenDataType::maxPrecision(), scale); +} + + template MutableColumnPtr DataTypeDecimal::createColumn() const { diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index 3f93f5aaae1..125f14cbf0a 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -103,6 +103,8 @@ public: void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; Field getDefault() const override; + bool canBeWiden() const override { return true; } + DataTypePtr getWidenDataType() const override; MutableColumnPtr createColumn() const override; bool equals(const IDataType & rhs) const override; diff --git a/dbms/src/DataTypes/DataTypesNumber.h b/dbms/src/DataTypes/DataTypesNumber.h index 6ce1cbc0d24..0794af41f36 100644 --- a/dbms/src/DataTypes/DataTypesNumber.h +++ b/dbms/src/DataTypes/DataTypesNumber.h @@ -17,6 +17,13 @@ class DataTypeNumber final : public DataTypeNumberBase bool canBeUsedInBitOperations() const override { return true; } bool canBeUsedInBooleanContext() const override { return true; } bool canBeInsideNullable() const override { return true; } + + bool canBeWiden() const override { return true; } + DataTypePtr getWidenDataType() const override + { + using WidenDataType = DataTypeNumber>; + return std::make_shared(); + } }; using DataTypeUInt8 = DataTypeNumber; diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index e8c6a3cc63d..4ebe1eb3e48 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -19,6 +19,7 @@ namespace ErrorCodes { extern const int MULTIPLE_STREAMS_REQUIRED; extern const int LOGICAL_ERROR; + extern const int DATA_TYPE_CANNOT_BE_WIDEN; } @@ -51,6 +52,10 @@ ColumnPtr IDataType::createColumnConstWithDefaultValue(size_t size) const return createColumnConst(size, getDefault()); } +DataTypePtr IDataType::getWidenDataType() const +{ + throw Exception("Data type " + getName() + " can't be widen.", ErrorCodes::DATA_TYPE_CANNOT_BE_WIDEN); +} void IDataType::serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const { diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index fdac4e454bc..3c20b258d09 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -273,6 +273,15 @@ public: */ virtual Field getDefault() const = 0; + /** The data type can be widen in order to try to avoid overflows. + * Widenable data types are typically Number or Decimal data types. + */ + virtual bool canBeWiden() const { return false; } + + /** Return the widen data type of the current data type. Throw an exception if `canBeWiden() == false`. + */ + virtual DataTypePtr getWidenDataType() const; + /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault. * This should be overriden if data type default value differs from column default value (example: Enum data types). */ From 0d6094a3eae8c0268fa66064bf92088dfe2db322 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 14:08:16 +0100 Subject: [PATCH 063/324] sumMap: return types less prone to oveflows It used to be that sumMap would return the same type as the values columns. If columns of Array(UInt8) were to be given, that would really easily cause oveflow. It now uses `getWidenDataType` (and ultimately `NearestFieldType`) in order to define the result type. --- .../src/AggregateFunctions/AggregateFunctionSumMap.h | 12 +++++++++++- .../queries/0_stateless/00502_sum_map.reference | 1 + dbms/tests/queries/0_stateless/00502_sum_map.sql | 9 +++++++++ 3 files changed, 21 insertions(+), 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 1e5f3e38cd2..8c7c24faed5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -72,7 +72,7 @@ public: types.emplace_back(std::make_shared(keys_type)); for (const auto & value_type : values_types) - types.emplace_back(std::make_shared(value_type)); + types.emplace_back(std::make_shared(widenDataType(value_type))); return std::make_shared(types); } @@ -260,6 +260,16 @@ public: const char * getHeaderFilePath() const override { return __FILE__; } bool keepKey(const T & key) const { return static_cast(*this).keepKey(key); } + +private: + static DataTypePtr widenDataType(const DataTypePtr & data_type) + { + if (!data_type->canBeWiden()) + throw new Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return data_type->getWidenDataType(); + } }; template diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.reference b/dbms/tests/queries/0_stateless/00502_sum_map.reference index 7bb325be814..a8d9fe95af3 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.reference +++ b/dbms/tests/queries/0_stateless/00502_sum_map.reference @@ -10,6 +10,7 @@ 2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10] ([1],[10]) ([1,4,8],[10,20,10]) +([1],[257]) ([1],[1]) ([1],[1]) (['a'],[1]) diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.sql b/dbms/tests/queries/0_stateless/00502_sum_map.sql index 9cf941dd908..24eab44d3d0 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.sql +++ b/dbms/tests/queries/0_stateless/00502_sum_map.sql @@ -17,6 +17,15 @@ SELECT sumMapFiltered([1, 4, 8])(statusMap.status, statusMap.requests) FROM test DROP TABLE test.sum_map; +DROP TABLE IF EXISTS test.sum_map_overflow; +CREATE TABLE test.sum_map_overflow(events Array(UInt8), counts Array(UInt8)) ENGINE = Log; + +INSERT INTO test.sum_map_overflow VALUES ([1], [255]), ([1], [2]); + +SELECT sumMap(events, counts) FROM test.sum_map_overflow; + +DROP TABLE test.sum_map_overflow; + select sumMap(val, cnt) from ( SELECT [ CAST(1, 'UInt64') ] as val, [1] as cnt ); select sumMap(val, cnt) from ( SELECT [ CAST(1, 'Float64') ] as val, [1] as cnt ); select sumMap(val, cnt) from ( SELECT [ CAST('a', 'Enum16(\'a\'=1)') ] as val, [1] as cnt ); From 468f8b4cfb6ce248accefa81d41fb1c14c8f5a4a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Jan 2019 17:08:01 +0300 Subject: [PATCH 064/324] Addition to prev. revision #4150 --- dbms/src/Interpreters/ThreadStatusExt.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/ThreadStatusExt.cpp b/dbms/src/Interpreters/ThreadStatusExt.cpp index 20a58de98f8..987365cb3c4 100644 --- a/dbms/src/Interpreters/ThreadStatusExt.cpp +++ b/dbms/src/Interpreters/ThreadStatusExt.cpp @@ -87,8 +87,8 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool if (!global_context) global_context = thread_group->global_context; - if (!thread_group->thread_statuses.emplace(thread_number, this).second) - throw Exception("Thread " + std::to_string(thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR); + /// NOTE: A thread may be attached multiple times if it is reused from a thread pool. + thread_group->thread_statuses.emplace(thread_number, this); } initPerformanceCounters(); From ed1136e49e21740002ef16bfe9476e3816761915 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 25 Jan 2019 17:15:44 +0300 Subject: [PATCH 065/324] Use more IO/WriteHelpers.h functions in the class ProtobufSimpleWriter, rename variant=>varint. --- dbms/src/Formats/ProtobufSimpleWriter.cpp | 159 ++++++------------ dbms/src/Formats/ProtobufSimpleWriter.h | 2 +- .../00825_protobuf_format_output.reference | Bin 2019 -> 2042 bytes .../00825_protobuf_format_output.sh | 2 +- .../ProtobufDelimitedMessagesSerializer.cpp | 22 +-- 5 files changed, 68 insertions(+), 117 deletions(-) diff --git a/dbms/src/Formats/ProtobufSimpleWriter.cpp b/dbms/src/Formats/ProtobufSimpleWriter.cpp index dce58f889f1..3959f0e09b5 100644 --- a/dbms/src/Formats/ProtobufSimpleWriter.cpp +++ b/dbms/src/Formats/ProtobufSimpleWriter.cpp @@ -1,15 +1,15 @@ #include #include -#include - +#include namespace DB { namespace { - void writeBytes(WriteBuffer & buf, const void * data, size_t size) { buf.write(reinterpret_cast(data), size); } - - void writeVariant(WriteBuffer & buf, UInt32 value) + // Note: We cannot simply use writeVarUInt() from IO/VarInt.h here because there is one small difference: + // Google protobuf's representation of 64-bit integer contains from 1 to 10 bytes, whileas writeVarUInt() writes from 1 to 9 bytes + // because it omits the tenth byte (which is not necessary to decode actually). + void writePbVarUInt(UInt64 value, WriteBuffer & buf) { while (value >= 0x80) { @@ -19,67 +19,18 @@ namespace buf.write(static_cast(value)); } - void writeVariant(WriteBuffer & buf, Int32 value) { writeVariant(buf, static_cast(value)); } - - void writeVariant(WriteBuffer & buf, UInt64 value) + void writePbVarInt(Int64 value, WriteBuffer & buf) { - while (value >= 0x80) - { - buf.write(static_cast(value | 0x80)); - value >>= 7; - } - buf.write(static_cast(value)); + writePbVarUInt((static_cast(value) << 1) ^ static_cast(value >> 63), buf); } - void writeVariant(WriteBuffer & buf, Int64 value) { writeVariant(buf, static_cast(value)); } - - void writeLittleEndian(WriteBuffer & buf, UInt32 value) - { - value = Poco::ByteOrder::toLittleEndian(value); - writeBytes(buf, &value, sizeof(value)); - } - - void writeLittleEndian(WriteBuffer & buf, Int32 value) { writeLittleEndian(buf, static_cast(value)); } - - void writeLittleEndian(WriteBuffer & buf, float value) - { - union - { - Float32 f; - UInt32 i; - }; - f = value; - writeLittleEndian(buf, i); - } - - void writeLittleEndian(WriteBuffer & buf, UInt64 value) - { - value = Poco::ByteOrder::toLittleEndian(value); - writeBytes(buf, &value, sizeof(value)); - } - - void writeLittleEndian(WriteBuffer & buf, Int64 value) { writeLittleEndian(buf, static_cast(value)); } - - void writeLittleEndian(WriteBuffer & buf, double value) - { - union - { - Float64 f; - UInt64 i; - }; - f = value; - writeLittleEndian(buf, i); - } - - UInt32 zigZag(Int32 value) { return (static_cast(value) << 1) ^ static_cast(value >> 31); } - UInt64 zigZag(Int64 value) { return (static_cast(value) << 1) ^ static_cast(value >> 63); } - + void writePbVarIntNoZigZagEncoding(Int64 value, WriteBuffer & buf) { writePbVarUInt(static_cast(value), buf); } } enum ProtobufSimpleWriter::WireType : UInt32 { - VARIANT = 0, + VARINT = 0, BITS64 = 1, LENGTH_DELIMITED = 2, BITS32 = 5 @@ -107,8 +58,8 @@ void ProtobufSimpleWriter::finishCurrentMessage() finishCurrentField(); current_field_number = 0; StringRef str = message_buffer.stringRef(); - writeVariant(out, str.size); - writeBytes(out, str.data, str.size); + writePbVarUInt(str.size, out); + out.write(str.data, str.size); message_buffer.restart(); } @@ -129,112 +80,112 @@ void ProtobufSimpleWriter::finishCurrentField() StringRef str = repeated_packing_buffer.stringRef(); if (str.size) { - writeKey(message_buffer, LENGTH_DELIMITED); - writeVariant(message_buffer, str.size); - writeBytes(message_buffer, str.data, str.size); + writeKey(LENGTH_DELIMITED, message_buffer); + writePbVarUInt(str.size, message_buffer); + message_buffer.write(str.data, str.size); repeated_packing_buffer.restart(); } } } -void ProtobufSimpleWriter::writeKey(WriteBuffer & buf, WireType wire_type) +void ProtobufSimpleWriter::writeKey(WireType wire_type, WriteBuffer & buf) { - writeVariant(buf, (current_field_number << 3) | wire_type); + writePbVarUInt((current_field_number << 3) | wire_type, buf); } void ProtobufSimpleWriter::writeInt32(Int32 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, value); + writeKey(VARINT, message_buffer); + writePbVarIntNoZigZagEncoding(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeUInt32(UInt32 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, value); + writeKey(VARINT, message_buffer); + writePbVarUInt(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeSInt32(Int32 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, zigZag(value)); + writeKey(VARINT, message_buffer); + writePbVarInt(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeInt64(Int64 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, value); + writeKey(VARINT, message_buffer); + writePbVarIntNoZigZagEncoding(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeUInt64(UInt64 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, value); + writeKey(VARINT, message_buffer); + writePbVarUInt(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeSInt64(Int64 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, zigZag(value)); + writeKey(VARINT, message_buffer); + writePbVarInt(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeFixed32(UInt32 value) { assert(current_field_number); - writeKey(message_buffer, BITS32); - writeLittleEndian(message_buffer, value); + writeKey(BITS32, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeSFixed32(Int32 value) { assert(current_field_number); - writeKey(message_buffer, BITS32); - writeLittleEndian(message_buffer, value); + writeKey(BITS32, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeFloat(float value) { assert(current_field_number); - writeKey(message_buffer, BITS32); - writeLittleEndian(message_buffer, value); + writeKey(BITS32, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeFixed64(UInt64 value) { assert(current_field_number); - writeKey(message_buffer, BITS64); - writeLittleEndian(message_buffer, value); + writeKey(BITS64, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeSFixed64(Int64 value) { assert(current_field_number); - writeKey(message_buffer, BITS64); - writeLittleEndian(message_buffer, value); + writeKey(BITS64, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeDouble(double value) { assert(current_field_number); - writeKey(message_buffer, BITS64); - writeLittleEndian(message_buffer, value); + writeKey(BITS64, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } @@ -242,9 +193,9 @@ void ProtobufSimpleWriter::writeString(const StringRef & str) { assert(current_field_number); ++num_normal_values; - writeKey(message_buffer, LENGTH_DELIMITED); - writeVariant(message_buffer, str.size); - writeBytes(message_buffer, str.data, str.size); + writeKey(LENGTH_DELIMITED, message_buffer); + writePbVarUInt(str.size, message_buffer); + message_buffer.write(str.data, str.size); } void ProtobufSimpleWriter::writeInt32IfNonZero(Int32 value) @@ -328,84 +279,84 @@ void ProtobufSimpleWriter::writeStringIfNotEmpty(const StringRef & str) void ProtobufSimpleWriter::packRepeatedInt32(Int32 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, value); + writePbVarIntNoZigZagEncoding(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedUInt32(UInt32 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, value); + writePbVarUInt(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedSInt32(Int32 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, zigZag(value)); + writePbVarInt(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedInt64(Int64 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, value); + writePbVarIntNoZigZagEncoding(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedUInt64(UInt64 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, value); + writePbVarUInt(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedSInt64(Int64 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, zigZag(value)); + writePbVarInt(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedFixed32(UInt32 value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedSFixed32(Int32 value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedFloat(float value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedFixed64(UInt64 value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedSFixed64(Int64 value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedDouble(double value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } diff --git a/dbms/src/Formats/ProtobufSimpleWriter.h b/dbms/src/Formats/ProtobufSimpleWriter.h index 4880cfb4e21..185fc1c2289 100644 --- a/dbms/src/Formats/ProtobufSimpleWriter.h +++ b/dbms/src/Formats/ProtobufSimpleWriter.h @@ -74,7 +74,7 @@ private: void finishCurrentField(); enum WireType : UInt32; - void writeKey(WriteBuffer & buf, WireType wire_type); + void writeKey(WireType wire_type, WriteBuffer & buf); WriteBuffer & out; bool were_messages = false; diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference index ed3e22f41e33b807ae354cd7045e96b86eae973f..d66a039f65c2aab60fa84126a7d0559dd4f4d800 100644 GIT binary patch delta 147 zcmaFN|BHXZTE@DG>wJVc0&*&I^Gi!IsyMRddOErFv>cc$%PzlJgVCOmsfBTJ29p7= z(F(Rd|Dk|!^BkrVj7(b?C$6)cJfFo#9$^5dsezt}fw8%fNfoEAiJqZ}k)e_C<~J-C Q7@3;E%D=Hm;IQu)0FNj)asU7T delta 125 zcmeyx|CoQmTE?o$7uYN(J{RU^2`Ni0s^Z92jCKmsV{qEc%NWPVSUowH$$-Ud1)IX= zRZJ%s88=Km&t^4wEsK#fL^-FaiJrNIk(se+6{oS0o`H#}k*VS4e=HXm8S8-t2cnxg MnU_(1axnXH0P?sdW&i*H diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh index a89126a3af8..ab021017e2b 100755 --- a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh @@ -44,7 +44,7 @@ CREATE TABLE test.table (uuid UUID, ) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO test.table VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000); -INSERT INTO test.table VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Tver', [54.782635, 32.045251], 3.14159, NULL, 0.007, 5.4, -20000000000000); +INSERT INTO test.table VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Plymouth', [50.403724, -4.142123], 3.14159, NULL, 0.007, 5.4, -20000000000000); INSERT INTO test.table VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000); SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_copy:Person'; diff --git a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp index 4a5596cb7f2..39877956686 100644 --- a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp +++ b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp @@ -53,9 +53,9 @@ int main(int, char **) person.add_color(100); person.add_color(200); person.add_color(50); - person.set_hometown("Tver"); - person.add_location(54.782635); - person.add_location(32.045251); + person.set_hometown("Plymouth"); + person.add_location(50.403724); + person.add_location(-4.142123); person.set_pi(3.14159); person.set_someratio(0.007); person.set_temperature(5.4); @@ -120,8 +120,8 @@ int main(int, char **) { AltPerson person; - person.add_location(54); - person.add_location(32); + person.add_location(50); + person.add_location(-4); person.set_pi(3.14159); person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); person.set_name("Natalia"); @@ -208,9 +208,9 @@ int main(int, char **) person.add_color("100"); person.add_color("200"); person.add_color("50"); - person.set_hometown("Tver"); - person.add_location("54.782635"); - person.add_location("32.045251"); + person.set_hometown("Plymouth"); + person.add_location("50.403724"); + person.add_location("-4.142123"); person.set_pi("3.14159"); person.set_someratio("0.007"); person.set_temperature("5.4"); @@ -290,9 +290,9 @@ int main(int, char **) person.add_color(100); person.add_color(200); person.add_color(50); - person.set_hometown("Tver"); - person.add_location(54.782635); - person.add_location(32.045251); + person.set_hometown("Plymouth"); + person.add_location(50.403724); + person.add_location(-4.142123); person.set_pi(3.14159); person.set_someratio(0.007); person.set_temperature(5.4); From 57104f81da213e95afb4623e5d416b0f8e1f3bb6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 15:16:23 +0100 Subject: [PATCH 066/324] rename `widen` to `promote` for IDataType --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.h | 8 ++++---- dbms/src/Common/ErrorCodes.cpp | 2 +- dbms/src/DataTypes/DataTypesDecimal.cpp | 6 +++--- dbms/src/DataTypes/DataTypesDecimal.h | 4 ++-- dbms/src/DataTypes/DataTypesNumber.h | 8 ++++---- dbms/src/DataTypes/IDataType.cpp | 6 +++--- dbms/src/DataTypes/IDataType.h | 10 +++++----- 7 files changed, 22 insertions(+), 22 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 8c7c24faed5..a1be9fa2a86 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -72,7 +72,7 @@ public: types.emplace_back(std::make_shared(keys_type)); for (const auto & value_type : values_types) - types.emplace_back(std::make_shared(widenDataType(value_type))); + types.emplace_back(std::make_shared(promoteNumericType(value_type))); return std::make_shared(types); } @@ -262,13 +262,13 @@ public: bool keepKey(const T & key) const { return static_cast(*this).keepKey(key); } private: - static DataTypePtr widenDataType(const DataTypePtr & data_type) + static DataTypePtr promoteNumericType(const DataTypePtr & data_type) { - if (!data_type->canBeWiden()) + if (!data_type->canBePromoted()) throw new Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - return data_type->getWidenDataType(); + return data_type->promoteNumericType(); } }; diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index d8749c1b1e7..574360d6979 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -412,7 +412,7 @@ namespace ErrorCodes extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD = 435; extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE = 436; extern const int PROTOBUF_FIELD_NOT_REPEATED = 437; - extern const int DATA_TYPE_CANNOT_BE_WIDEN = 438; + extern const int DATA_TYPE_CANNOT_BE_PROMOTED = 438; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index f8f3084b254..e6fe1bbd7f2 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -149,10 +149,10 @@ Field DataTypeDecimal::getDefault() const template -DataTypePtr DataTypeDecimal::getWidenDataType() const +DataTypePtr DataTypeDecimal::promoteNumericType() const { - using WidenDataType = DataTypeDecimal; - return std::make_shared(WidenDataType::maxPrecision(), scale); + using PromotedType = DataTypeDecimal; + return std::make_shared(PromotedType::maxPrecision(), scale); } diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index 125f14cbf0a..c287742928a 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -103,8 +103,8 @@ public: void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; Field getDefault() const override; - bool canBeWiden() const override { return true; } - DataTypePtr getWidenDataType() const override; + bool canBePromoted() const override { return true; } + DataTypePtr promoteNumericType() const override; MutableColumnPtr createColumn() const override; bool equals(const IDataType & rhs) const override; diff --git a/dbms/src/DataTypes/DataTypesNumber.h b/dbms/src/DataTypes/DataTypesNumber.h index 0794af41f36..f54e014dfc5 100644 --- a/dbms/src/DataTypes/DataTypesNumber.h +++ b/dbms/src/DataTypes/DataTypesNumber.h @@ -18,11 +18,11 @@ class DataTypeNumber final : public DataTypeNumberBase bool canBeUsedInBooleanContext() const override { return true; } bool canBeInsideNullable() const override { return true; } - bool canBeWiden() const override { return true; } - DataTypePtr getWidenDataType() const override + bool canBePromoted() const override { return true; } + DataTypePtr promoteNumericType() const override { - using WidenDataType = DataTypeNumber>; - return std::make_shared(); + using PromotedType = DataTypeNumber>; + return std::make_shared(); } }; diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index 4ebe1eb3e48..a8eec08a7c8 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -19,7 +19,7 @@ namespace ErrorCodes { extern const int MULTIPLE_STREAMS_REQUIRED; extern const int LOGICAL_ERROR; - extern const int DATA_TYPE_CANNOT_BE_WIDEN; + extern const int DATA_TYPE_CANNOT_BE_PROMOTED; } @@ -52,9 +52,9 @@ ColumnPtr IDataType::createColumnConstWithDefaultValue(size_t size) const return createColumnConst(size, getDefault()); } -DataTypePtr IDataType::getWidenDataType() const +DataTypePtr IDataType::promoteNumericType() const { - throw Exception("Data type " + getName() + " can't be widen.", ErrorCodes::DATA_TYPE_CANNOT_BE_WIDEN); + throw Exception("Data type " + getName() + " can't be promoted.", ErrorCodes::DATA_TYPE_CANNOT_BE_PROMOTED); } void IDataType::serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 3c20b258d09..1fce04639bf 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -273,14 +273,14 @@ public: */ virtual Field getDefault() const = 0; - /** The data type can be widen in order to try to avoid overflows. - * Widenable data types are typically Number or Decimal data types. + /** The data type can be promoted in order to try to avoid overflows. + * Data types which can be promoted are typically Number or Decimal data types. */ - virtual bool canBeWiden() const { return false; } + virtual bool canBePromoted() const { return false; } - /** Return the widen data type of the current data type. Throw an exception if `canBeWiden() == false`. + /** Return the promoted numeric data type of the current data type. Throw an exception if `canBePromoted() == false`. */ - virtual DataTypePtr getWidenDataType() const; + virtual DataTypePtr promoteNumericType() const; /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault. * This should be overriden if data type default value differs from column default value (example: Enum data types). From a78282b8347704d1456f9b1e66383347def5b094 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 15:42:44 +0100 Subject: [PATCH 067/324] IDataType.h: remove trailing whitespace --- dbms/src/DataTypes/IDataType.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 1fce04639bf..ae14fe1aa36 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -280,7 +280,7 @@ public: /** Return the promoted numeric data type of the current data type. Throw an exception if `canBePromoted() == false`. */ - virtual DataTypePtr promoteNumericType() const; + virtual DataTypePtr promoteNumericType() const; /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault. * This should be overriden if data type default value differs from column default value (example: Enum data types). From d8eba7e71b95d617ae5df1865d2c397c6e69a635 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 25 Jan 2019 22:50:31 +0800 Subject: [PATCH 068/324] Better joinGet. --- dbms/src/Functions/FunctionJoinGet.cpp | 51 ++++++++++++++----- dbms/src/Functions/FunctionJoinGet.h | 21 ++++++-- .../00800_versatile_storage_join.reference | 2 + .../00800_versatile_storage_join.sql | 13 +++++ 4 files changed, 71 insertions(+), 16 deletions(-) diff --git a/dbms/src/Functions/FunctionJoinGet.cpp b/dbms/src/Functions/FunctionJoinGet.cpp index d2f65148b23..1c0cbff7506 100644 --- a/dbms/src/Functions/FunctionJoinGet.cpp +++ b/dbms/src/Functions/FunctionJoinGet.cpp @@ -1,10 +1,10 @@ #include +#include #include #include #include #include -#include namespace DB @@ -15,10 +15,10 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -FunctionBasePtr FunctionBuilderJoinGet::buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const +static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) { if (arguments.size() != 3) - throw Exception{"Function " + getName() + " takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + throw Exception{"Function joinGet takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; String join_name; if (auto name_col = checkAndGetColumnConst(arguments[0].column.get())) @@ -26,37 +26,62 @@ FunctionBasePtr FunctionBuilderJoinGet::buildImpl(const ColumnsWithTypeAndName & join_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() - + ", expected a const string.", + throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - auto table = context.getTable("", join_name); - - StorageJoin * storage_join = dynamic_cast(table.get()); - + size_t dot = join_name.find('.'); + String database_name; + if (dot == String::npos) + { + database_name = context.getCurrentDatabase(); + dot = 0; + } + else + { + database_name = join_name.substr(0, dot); + ++dot; + } + String table_name = join_name.substr(dot); + auto table = context.getTable(database_name, table_name); + auto storage_join = std::dynamic_pointer_cast(table); if (!storage_join) throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - auto join = storage_join->getJoin(); String attr_name; if (auto name_col = checkAndGetColumnConst(arguments[1].column.get())) { attr_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() - + ", expected a const string.", + throw Exception{"Illegal type " + arguments[1].type->getName() + + " of second argument of function joinGet, expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + return std::make_pair(storage_join, attr_name); +} +FunctionBasePtr FunctionBuilderJoinGet::buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const +{ + auto [storage_join, attr_name] = getJoin(arguments, context); + auto join = storage_join->getJoin(); DataTypes data_types(arguments.size()); + auto table_lock = storage_join->lockStructure(false); for (size_t i = 0; i < arguments.size(); ++i) data_types[i] = arguments[i].type; + auto return_type = join->joinGetReturnType(attr_name); return std::make_shared( - std::make_shared(join, attr_name), data_types, join->joinGetReturnType(attr_name)); + std::make_shared(table_lock, storage_join, join, attr_name, return_type), data_types, return_type); } +DataTypePtr FunctionBuilderJoinGet::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + auto [storage_join, attr_name] = getJoin(arguments, context); + auto join = storage_join->getJoin(); + return join->joinGetReturnType(attr_name); +} + + void FunctionJoinGet::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) { auto & ctn = block.getByPosition(arguments[2]); diff --git a/dbms/src/Functions/FunctionJoinGet.h b/dbms/src/Functions/FunctionJoinGet.h index 6573dcf9405..edf45adab6a 100644 --- a/dbms/src/Functions/FunctionJoinGet.h +++ b/dbms/src/Functions/FunctionJoinGet.h @@ -3,20 +3,32 @@ namespace DB { class Context; +class IStorage; +using StoragePtr = std::shared_ptr; class Join; using JoinPtr = std::shared_ptr; +class TableStructureReadLock; +using TableStructureReadLockPtr = std::shared_ptr; class FunctionJoinGet final : public IFunction, public std::enable_shared_from_this { public: static constexpr auto name = "joinGet"; - FunctionJoinGet(JoinPtr join, const String & attr_name) : join(std::move(join)), attr_name(attr_name) {} + FunctionJoinGet( + TableStructureReadLockPtr table_lock, StoragePtr storage_join, JoinPtr join, const String & attr_name, DataTypePtr return_type) + : table_lock(std::move(table_lock)) + , storage_join(std::move(storage_join)) + , join(std::move(join)) + , attr_name(attr_name) + , return_type(std::move(return_type)) + { + } String getName() const override { return name; } protected: - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return nullptr; } + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return return_type; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; private: @@ -24,8 +36,11 @@ private: size_t getNumberOfArguments() const override { return 0; } private: + TableStructureReadLockPtr table_lock; + StoragePtr storage_join; JoinPtr join; const String attr_name; + DataTypePtr return_type; }; class FunctionBuilderJoinGet final : public FunctionBuilderImpl @@ -40,7 +55,7 @@ public: protected: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override; - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return nullptr; } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; private: bool isVariadic() const override { return true; } diff --git a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.reference b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.reference index 1fa9ac74e57..9b706623c8c 100644 --- a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.reference +++ b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.reference @@ -17,3 +17,5 @@ abc def [0] 1 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql index 80f7616766c..47c14765ba9 100644 --- a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql +++ b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql @@ -49,3 +49,16 @@ DROP TABLE test.join_any_left_null; DROP TABLE test.join_all_inner; DROP TABLE test.join_all_left; DROP TABLE test.join_string_key; + +-- test provided by Alexander Zaitsev +DROP TABLE IF EXISTS test.join_test; +CREATE TABLE test.join_test (a UInt8, b UInt8) Engine = Join(ANY, LEFT, a); + +USE test; +select joinGet('join_test', 'b', 1); + +USE system; +SELECT joinGet('test.join_test', 'b', 1); + +USE default; +DROP TABLE test.join_test; From 091efcda9553610818c00088b93e903ed1ed67cf Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 25 Jan 2019 18:42:24 +0300 Subject: [PATCH 069/324] add IdentifierSemantic and PredicateExpressionsOptimizer refactoring --- .../Interpreters/AddDefaultDatabaseVisitor.h | 3 +- .../Interpreters/ArrayJoinedColumnsVisitor.h | 3 +- dbms/src/Interpreters/ColumnNamesContext.cpp | 3 +- .../DatabaseAndTableWithAlias.cpp | 68 +---- .../Interpreters/DatabaseAndTableWithAlias.h | 10 +- dbms/src/Interpreters/ExternalTablesVisitor.h | 3 +- .../Interpreters/GlobalSubqueriesVisitor.h | 1 + dbms/src/Interpreters/IdentifierSemantic.cpp | 106 +++++++ dbms/src/Interpreters/IdentifierSemantic.h | 35 +++ .../InJoinSubqueriesPreprocessor.cpp | 1 + .../Interpreters/InterpreterDescribeQuery.cpp | 16 +- .../PredicateExpressionsOptimizer.cpp | 276 ++++++++++-------- .../PredicateExpressionsOptimizer.h | 48 ++- dbms/src/Interpreters/QueryNormalizer.cpp | 8 +- .../RequiredSourceColumnsVisitor.cpp | 3 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 35 +-- .../TranslateQualifiedNamesVisitor.cpp | 45 +-- .../TranslateQualifiedNamesVisitor.h | 2 +- dbms/src/Parsers/ASTIdentifier.cpp | 82 ++---- dbms/src/Parsers/ASTIdentifier.h | 54 ++-- .../MergeTree/MergeTreeWhereOptimizer.cpp | 3 +- dbms/src/Storages/StorageView.cpp | 1 + dbms/src/Storages/VirtualColumnUtils.cpp | 3 +- .../TableFunctions/TableFunctionRemote.cpp | 1 + .../00597_push_down_predicate.reference | 1 + .../0_stateless/00597_push_down_predicate.sql | 3 +- .../00674_join_on_syntax.reference | 4 +- .../0_stateless/00674_join_on_syntax.sql | 12 +- .../queries/0_stateless/00703_join_crash.sql | 2 +- 29 files changed, 433 insertions(+), 399 deletions(-) create mode 100644 dbms/src/Interpreters/IdentifierSemantic.cpp create mode 100644 dbms/src/Interpreters/IdentifierSemantic.h diff --git a/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h index 2b032ff8a88..ce542a63503 100644 --- a/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { @@ -99,7 +100,7 @@ private: /// @note It expects that only table (not column) identifiers are visited. void visit(const ASTIdentifier & identifier, ASTPtr & ast) const { - if (identifier.name_parts.empty()) + if (!identifier.compound()) ast = createTableIdentifier(database_name, identifier.name); } diff --git a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h index 0c19ffe7387..6aed7436572 100644 --- a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -10,6 +10,7 @@ #include #include +#include #include @@ -95,7 +96,7 @@ private: NameToNameMap & array_join_alias_to_name = data.array_join_alias_to_name; NameToNameMap & array_join_result_to_source = data.array_join_result_to_source; - if (!getColumnIdentifierName(node)) + if (!IdentifierSemantic::getColumnName(node)) return; auto splitted = Nested::splitName(node.name); /// ParsedParams, Key1 diff --git a/dbms/src/Interpreters/ColumnNamesContext.cpp b/dbms/src/Interpreters/ColumnNamesContext.cpp index 246b5f5306e..3240ca7fea1 100644 --- a/dbms/src/Interpreters/ColumnNamesContext.cpp +++ b/dbms/src/Interpreters/ColumnNamesContext.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB @@ -31,7 +32,7 @@ bool ColumnNamesContext::addColumnAliasIfAny(const IAST & ast, bool is_public) void ColumnNamesContext::addColumnIdentifier(const ASTIdentifier & node, bool is_public) { - if (!getColumnIdentifierName(node)) + if (!IdentifierSemantic::getColumnName(node)) return; required_names.insert(node.name); diff --git a/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp b/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp index efd29f962d0..c9afb5da722 100644 --- a/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp +++ b/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -11,60 +12,13 @@ namespace DB { -/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left. -/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'. -void stripIdentifier(const DB::ASTPtr & ast, size_t num_qualifiers_to_strip) -{ - ASTIdentifier * identifier = typeid_cast(ast.get()); - - if (!identifier) - throw DB::Exception("ASTIdentifier expected for stripIdentifier", DB::ErrorCodes::LOGICAL_ERROR); - - if (num_qualifiers_to_strip) - { - identifier->name_parts.erase(identifier->name_parts.begin(), identifier->name_parts.begin() + num_qualifiers_to_strip); - DB::String new_name; - for (const auto & part : identifier->name_parts) - { - if (!new_name.empty()) - new_name += '.'; - new_name += part; - } - identifier->name.swap(new_name); - } -} - -/// Get the number of components of identifier which are correspond to 'alias.', 'table.' or 'databas.table.' from names. -size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier, - const DatabaseAndTableWithAlias & names) -{ - /// database.table.column - if (doesIdentifierBelongTo(identifier, names.database, names.table)) - return 2; - - /// table.column or alias.column. - if (doesIdentifierBelongTo(identifier, names.table) || - doesIdentifierBelongTo(identifier, names.alias)) - return 1; - - return 0; -} - - DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database) { - database = current_database; - table = identifier.name; alias = identifier.tryGetAlias(); - if (!identifier.name_parts.empty()) - { - if (identifier.name_parts.size() != 2) - throw Exception("Logical error: 2 components expected in table expression '" + identifier.name + "'", ErrorCodes::LOGICAL_ERROR); - - database = identifier.name_parts[0]; - table = identifier.name_parts[1]; - } + std::tie(database, table) = IdentifierSemantic::extractDatabaseAndTable(identifier); + if (database.empty()) + database = current_database; } DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database) @@ -108,19 +62,7 @@ String DatabaseAndTableWithAlias::getQualifiedNamePrefix() const { if (alias.empty() && table.empty()) return ""; - - return (!alias.empty() ? alias : (database + '.' + table)) + '.'; -} - -void DatabaseAndTableWithAlias::makeQualifiedName(const ASTPtr & ast) const -{ - if (auto identifier = typeid_cast(ast.get())) - { - String prefix = getQualifiedNamePrefix(); - identifier->name.insert(identifier->name.begin(), prefix.begin(), prefix.end()); - - addIdentifierQualifier(*identifier, database, table, alias); - } + return (!alias.empty() ? alias : table) + '.'; } std::vector getSelectTablesExpression(const ASTSelectQuery & select_query) diff --git a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h index 87884e26273..bb4f7ca92ef 100644 --- a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h +++ b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h @@ -29,21 +29,13 @@ struct DatabaseAndTableWithAlias DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = ""); DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database); - /// "alias." or "database.table." if alias is empty + /// "alias." or "table." if alias is empty String getQualifiedNamePrefix() const; - /// If ast is ASTIdentifier, prepend getQualifiedNamePrefix() to it's name. - void makeQualifiedName(const ASTPtr & ast) const; - /// Check if it satisfies another db_table name. @note opterion is not symmetric. bool satisfies(const DatabaseAndTableWithAlias & table, bool table_may_be_an_alias); }; -void stripIdentifier(const DB::ASTPtr & ast, size_t num_qualifiers_to_strip); - -size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier, - const DatabaseAndTableWithAlias & names); - std::vector getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database); std::optional getDatabaseAndTable(const ASTSelectQuery & select, size_t table_number); diff --git a/dbms/src/Interpreters/ExternalTablesVisitor.h b/dbms/src/Interpreters/ExternalTablesVisitor.h index d8b177b1ed3..2f3eecd2828 100644 --- a/dbms/src/Interpreters/ExternalTablesVisitor.h +++ b/dbms/src/Interpreters/ExternalTablesVisitor.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -33,7 +34,7 @@ public: private: static std::vector visit(const ASTIdentifier & node, ASTPtr &, Data & data) { - if (auto opt_name = getTableIdentifierName(node)) + if (auto opt_name = IdentifierSemantic::getTableName(node)) if (StoragePtr external_storage = data.context.tryGetExternalTable(*opt_name)) data.external_tables[*opt_name] = external_storage; return {}; diff --git a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h index 3e4108464f9..81e45d2abea 100644 --- a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { diff --git a/dbms/src/Interpreters/IdentifierSemantic.cpp b/dbms/src/Interpreters/IdentifierSemantic.cpp new file mode 100644 index 00000000000..e6fe2257d20 --- /dev/null +++ b/dbms/src/Interpreters/IdentifierSemantic.cpp @@ -0,0 +1,106 @@ +#include + +#include + +namespace DB +{ + +std::optional IdentifierSemantic::getColumnName(const ASTIdentifier & node) +{ + if (!node.semantic->special) + return node.name; + return {}; +} + +std::optional IdentifierSemantic::getColumnName(const ASTPtr & ast) +{ + if (ast) + if (auto id = typeid_cast(ast.get())) + if (!id->semantic->special) + return id->name; + return {}; +} + +std::optional IdentifierSemantic::getTableName(const ASTIdentifier & node) +{ + if (node.semantic->special) + return node.name; + return {}; +} + +std::optional IdentifierSemantic::getTableName(const ASTPtr & ast) +{ + if (ast) + if (auto id = typeid_cast(ast.get())) + if (id->semantic->special) + return id->name; + return {}; +} + +std::pair IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier) +{ + if (identifier.name_parts.size() > 2) + throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR); + + if (identifier.name_parts.size() == 2) + return { identifier.name_parts[0], identifier.name_parts[1] }; + return { "", identifier.name }; +} + +bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table) +{ + size_t num_components = identifier.name_parts.size(); + if (num_components >= 3) + return identifier.name_parts[0] == database && + identifier.name_parts[1] == table; + return false; +} + +bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table) +{ + size_t num_components = identifier.name_parts.size(); + if (num_components >= 2) + return identifier.name_parts[0] == table; + return false; +} + +size_t IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table) +{ + /// database.table.column + if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table)) + return 2; + + /// table.column or alias.column. + if (doesIdentifierBelongTo(identifier, db_and_table.table) || + doesIdentifierBelongTo(identifier, db_and_table.alias)) + return 1; + + return 0; +} + +/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left. +/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'. +void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t to_strip) +{ + if (!to_strip) + return; + + std::vector stripped(identifier.name_parts.begin() + to_strip, identifier.name_parts.end()); + + DB::String new_name; + for (const auto & part : stripped) + { + if (!new_name.empty()) + new_name += '.'; + new_name += part; + } + identifier.name.swap(new_name); +} + +void IdentifierSemantic::setColumnQualifiedName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table) +{ + String prefix = db_and_table.getQualifiedNamePrefix(); + identifier.name.insert(identifier.name.begin(), prefix.begin(), prefix.end()); +} + +} diff --git a/dbms/src/Interpreters/IdentifierSemantic.h b/dbms/src/Interpreters/IdentifierSemantic.h new file mode 100644 index 00000000000..895a51899fe --- /dev/null +++ b/dbms/src/Interpreters/IdentifierSemantic.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct IdentifierSemanticImpl +{ + bool special = false; +}; + +/// Static calss to manipulate IdentifierSemanticImpl via ASTIdentifier +struct IdentifierSemantic +{ + /// @returns name for column identifiers + static std::optional getColumnName(const ASTIdentifier & node); + static std::optional getColumnName(const ASTPtr & ast); + + /// @returns name for 'not a column' identifiers + static std::optional getTableName(const ASTIdentifier & node); + static std::optional getTableName(const ASTPtr & ast); + static std::pair extractDatabaseAndTable(const ASTIdentifier & identifier); + + static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); + static void setColumnShortName(ASTIdentifier & identifier, size_t match); + static void setColumnQualifiedName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); + +private: + static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table); + static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table); +}; + +} diff --git a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index e8e697cfe4f..e0be0d068e0 100644 --- a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index 7c2812eb428..b80426fcdd9 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -83,20 +84,11 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() } else { + auto identifier = typeid_cast(table_expression->database_and_table_name.get()); + String database_name; String table_name; - - auto identifier = typeid_cast(table_expression->database_and_table_name.get()); - if (identifier->name_parts.size() > 2) - throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR); - - if (identifier->name_parts.size() > 1) - { - database_name = identifier->name_parts[0]; - table_name = identifier->name_parts[1]; - } - else - table_name = identifier->name; + std::tie(database_name, table_name) = IdentifierSemantic::extractDatabaseAndTable(*identifier); table = context.getTable(database_name, table_name); } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 99ea0479009..cd4c33ce558 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -1,12 +1,23 @@ +#include + #include #include #include #include +#include #include -#include +#include +#include +#include +#include +#include +#include +#include #include #include #include +#include +#include #include #include #include "TranslateQualifiedNamesVisitor.h" @@ -22,6 +33,59 @@ namespace ErrorCodes static constexpr auto and_function_name = "and"; + +struct FindIdentifierBestTableData +{ + using TypeToVisit = ASTIdentifier; + + const std::vector & tables; + std::vector> identifier_table; + + FindIdentifierBestTableData(const std::vector & tables_) + : tables(tables_) + {} + + void visit(ASTIdentifier & identifier, ASTPtr &) + { + const DatabaseAndTableWithAlias * best_table = nullptr; + + if (!identifier.compound()) + { + if (!tables.empty()) + best_table = &tables[0]; + } + else + { + size_t best_match = 0; + for (const DatabaseAndTableWithAlias & table : tables) + { + if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, table)) + if (match > best_match) + { + best_match = match; + best_table = &table; + } + } + } + + identifier_table.emplace_back(&identifier, best_table); + } +}; + +using FindIdentifierBestTableMatcher = OneTypeMatcher; +using FindIdentifierBestTableVisitor = InDepthNodeVisitor; + + +static bool allowPushDown(const ASTSelectQuery * subquery) +{ + return subquery && + !subquery->final() && + !subquery->limit_by_expression_list && + !subquery->limit_length && + !subquery->with_expression_list; +} + + PredicateExpressionsOptimizer::PredicateExpressionsOptimizer( ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_) : ast_select(ast_select_), settings(settings_), context(context_) @@ -36,47 +100,50 @@ bool PredicateExpressionsOptimizer::optimize() if (!ast_select->where_expression && !ast_select->prewhere_expression) return false; - SubqueriesProjectionColumns all_subquery_projection_columns; - getAllSubqueryProjectionColumns(all_subquery_projection_columns); + SubqueriesProjectionColumns all_subquery_projection_columns = getAllSubqueryProjectionColumns(); bool is_rewrite_subqueries = false; if (!all_subquery_projection_columns.empty()) { - is_rewrite_subqueries |= optimizeImpl(ast_select->where_expression, all_subquery_projection_columns, false); - is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere_expression, all_subquery_projection_columns, true); + is_rewrite_subqueries |= optimizeImpl(ast_select->where_expression, all_subquery_projection_columns, OptimizeKind::PUSH_TO_WHERE); + is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere_expression, all_subquery_projection_columns, OptimizeKind::PUSH_TO_PREWHERE); } return is_rewrite_subqueries; } bool PredicateExpressionsOptimizer::optimizeImpl( - ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, bool is_prewhere) + ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind) { /// split predicate with `and` - PredicateExpressions outer_predicate_expressions = splitConjunctionPredicate(outer_expression); + std::vector outer_predicate_expressions = splitConjunctionPredicate(outer_expression); std::vector database_and_table_with_aliases = getDatabaseAndTables(*ast_select, context.getCurrentDatabase()); bool is_rewrite_subquery = false; - for (const auto & outer_predicate : outer_predicate_expressions) + for (auto & outer_predicate : outer_predicate_expressions) { if (isArrayJoinFunction(outer_predicate)) continue; - IdentifiersWithQualifiedNameSet outer_predicate_dependencies; - getDependenciesAndQualifiedOfExpression(outer_predicate, outer_predicate_dependencies, database_and_table_with_aliases); + auto outer_predicate_dependencies = getDependenciesAndQualifiers(outer_predicate, database_and_table_with_aliases); /// TODO: remove origin expression - for (const auto & subquery_projection_columns : subqueries_projection_columns) + for (const auto & [subquery, projection_columns] : subqueries_projection_columns) { - auto subquery = static_cast(subquery_projection_columns.first); - const ProjectionsWithAliases projection_columns = subquery_projection_columns.second; - OptimizeKind optimize_kind = OptimizeKind::NONE; - if (!cannotPushDownOuterPredicate(projection_columns, subquery, outer_predicate_dependencies, is_prewhere, optimize_kind)) + if (allowPushDown(subquery) && canPushDownOuterPredicate(projection_columns, outer_predicate_dependencies, optimize_kind)) { - ASTPtr inner_predicate; - cloneOuterPredicateForInnerPredicate(outer_predicate, projection_columns, database_and_table_with_aliases, inner_predicate); + if (optimize_kind == OptimizeKind::NONE) + optimize_kind = expression_kind; + + ASTPtr inner_predicate = outer_predicate->clone(); + cleanExpressionAlias(inner_predicate); /// clears the alias name contained in the outer predicate + + std::vector inner_predicate_dependencies = + getDependenciesAndQualifiers(inner_predicate, database_and_table_with_aliases); + + setNewAliasesForInnerPredicate(projection_columns, inner_predicate_dependencies); switch (optimize_kind) { @@ -91,9 +158,9 @@ bool PredicateExpressionsOptimizer::optimizeImpl( return is_rewrite_subquery; } -PredicateExpressions PredicateExpressionsOptimizer::splitConjunctionPredicate(ASTPtr & predicate_expression) +std::vector PredicateExpressionsOptimizer::splitConjunctionPredicate(ASTPtr & predicate_expression) { - PredicateExpressions predicate_expressions; + std::vector predicate_expressions; if (predicate_expression) { @@ -127,77 +194,79 @@ PredicateExpressions PredicateExpressionsOptimizer::splitConjunctionPredicate(AS return predicate_expressions; } -void PredicateExpressionsOptimizer::getDependenciesAndQualifiedOfExpression(const ASTPtr & expression, - IdentifiersWithQualifiedNameSet & dependencies_and_qualified, - std::vector & tables_with_aliases) +std::vector +PredicateExpressionsOptimizer::getDependenciesAndQualifiers(ASTPtr & expression, std::vector & tables) { - if (const auto identifier = typeid_cast(expression.get())) + FindIdentifierBestTableVisitor::Data find_data(tables); + FindIdentifierBestTableVisitor(find_data).visit(expression); + + std::vector dependencies; + + for (const auto & [identifier, table] : find_data.identifier_table) { String table_alias; - if (!identifier->name_parts.empty()) - { - if (!tables_with_aliases.empty()) - table_alias = tables_with_aliases[0].getQualifiedNamePrefix(); - } - else - { - size_t best_table_pos = 0; - size_t max_num_qualifiers_to_strip = 0; + if (table) + table_alias = table->getQualifiedNamePrefix(); - /// translate qualifiers for dependent columns - for (size_t table_pos = 0; table_pos < tables_with_aliases.size(); ++table_pos) - { - const auto & table = tables_with_aliases[table_pos]; - auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, table); - - if (num_qualifiers_to_strip > max_num_qualifiers_to_strip) - { - max_num_qualifiers_to_strip = num_qualifiers_to_strip; - best_table_pos = table_pos; - } - } - - table_alias = tables_with_aliases[best_table_pos].getQualifiedNamePrefix(); - } - - String qualified_name = table_alias + expression->getAliasOrColumnName(); - dependencies_and_qualified.emplace_back(std::pair(identifier, qualified_name)); - } - else - { - for (const auto & child : expression->children) - getDependenciesAndQualifiedOfExpression(child, dependencies_and_qualified, tables_with_aliases); + dependencies.emplace_back(identifier, table_alias); } + + return dependencies; } -bool PredicateExpressionsOptimizer::cannotPushDownOuterPredicate( - const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery, - IdentifiersWithQualifiedNameSet & outer_predicate_dependencies, bool & is_prewhere, OptimizeKind & optimize_kind) +static String qualifiedName(ASTIdentifier * identifier, const String & prefix) { - if (subquery->final() || subquery->limit_by_expression_list || subquery->limit_length || subquery->with_expression_list) - return true; + if (identifier->isShort()) + return prefix + identifier->getAliasOrColumnName(); + return identifier->getAliasOrColumnName(); +} - for (auto & predicate_dependency : outer_predicate_dependencies) +bool PredicateExpressionsOptimizer::canPushDownOuterPredicate( + const std::vector & projection_columns, + const std::vector & dependencies, + OptimizeKind & optimize_kind) +{ + for (const auto & [identifier, prefix] : dependencies) { bool is_found = false; + String qualified_name = qualifiedName(identifier, prefix); - for (auto projection_column : subquery_projection_columns) + for (const auto & [ast, alias] : projection_columns) { - if (projection_column.second == predicate_dependency.second) + if (alias == qualified_name) { is_found = true; - optimize_kind = isAggregateFunction(projection_column.first) ? OptimizeKind::PUSH_TO_HAVING : optimize_kind; + if (isAggregateFunction(ast)) + optimize_kind = OptimizeKind::PUSH_TO_HAVING; } } if (!is_found) - return true; + return false; } - if (optimize_kind == OptimizeKind::NONE) - optimize_kind = is_prewhere ? OptimizeKind::PUSH_TO_PREWHERE : OptimizeKind::PUSH_TO_WHERE; + return true; +} - return false; +void PredicateExpressionsOptimizer::setNewAliasesForInnerPredicate( + const std::vector & projection_columns, + const std::vector & dependencies) +{ + for (auto & [identifier, prefix] : dependencies) + { + String qualified_name = qualifiedName(identifier, prefix); + + for (auto & [ast, alias] : projection_columns) + { + if (alias == qualified_name) + { + if (!isIdentifier(ast) && ast->tryGetAlias().empty()) + ast->setAlias(ast->getColumnName()); + + identifier->resetWithAlias(ast->getAliasOrColumnName()); + } + } + } } bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node) @@ -215,47 +284,21 @@ bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node) return false; } -bool PredicateExpressionsOptimizer::isAggregateFunction(ASTPtr & node) +bool PredicateExpressionsOptimizer::isAggregateFunction(const ASTPtr & node) { - if (auto function = typeid_cast(node.get())) + if (auto function = typeid_cast(node.get())) { if (AggregateFunctionFactory::instance().isAggregateFunctionName(function->name)) return true; } - for (auto & child : node->children) + for (const auto & child : node->children) if (isAggregateFunction(child)) return true; return false; } -void PredicateExpressionsOptimizer::cloneOuterPredicateForInnerPredicate( - const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns, - std::vector & tables, ASTPtr & inner_predicate) -{ - inner_predicate = outer_predicate->clone(); - - /// clears the alias name contained in the outer predicate - cleanExpressionAlias(inner_predicate); - IdentifiersWithQualifiedNameSet new_expression_requires; - getDependenciesAndQualifiedOfExpression(inner_predicate, new_expression_requires, tables); - - for (auto & require : new_expression_requires) - { - for (auto projection : projection_columns) - { - if (require.second == projection.second) - { - ASTPtr & ast = projection.first; - if (!typeid_cast(ast.get()) && ast->tryGetAlias().empty()) - ast->setAlias(ast->getColumnName()); - require.first->name = ast->getAliasOrColumnName(); - } - } - } -} - bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery) { ASTPtr new_subquery_expression = subquery_expression; @@ -272,33 +315,32 @@ bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expr return true; } -void PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns) +PredicateExpressionsOptimizer::SubqueriesProjectionColumns PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns() { - const auto tables_expression = getSelectTablesExpression(*ast_select); + SubqueriesProjectionColumns projection_columns; - for (const auto & table_expression : tables_expression) - { + for (const auto & table_expression : getSelectTablesExpression(*ast_select)) if (table_expression->subquery) - { - /// Use qualifiers to translate the columns of subqueries - DatabaseAndTableWithAlias database_and_table_with_alias(*table_expression, context.getCurrentDatabase()); - String qualified_name_prefix = database_and_table_with_alias.getQualifiedNamePrefix(); - getSubqueryProjectionColumns(all_subquery_projection_columns, qualified_name_prefix, - static_cast(table_expression->subquery.get())->children[0]); - } - } + getSubqueryProjectionColumns(table_expression->subquery, projection_columns); + + return projection_columns; } -void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns, - String & qualified_name_prefix, const ASTPtr & subquery) +void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & projection_columns) { - ASTs select_with_union_projections; - auto select_with_union_query = static_cast(subquery.get()); + String qualified_name_prefix = subquery->tryGetAlias(); + if (!qualified_name_prefix.empty()) + qualified_name_prefix += '.'; - for (auto & select_without_union_query : select_with_union_query->list_of_selects->children) + const ASTPtr & subselect = subquery->children[0]; + + ASTs select_with_union_projections; + auto select_with_union_query = static_cast(subselect.get()); + + for (auto & select : select_with_union_query->list_of_selects->children) { - ProjectionsWithAliases subquery_projections; - auto select_projection_columns = getSelectQueryProjectionColumns(select_without_union_query); + std::vector subquery_projections; + auto select_projection_columns = getSelectQueryProjectionColumns(select); if (!select_projection_columns.empty()) { @@ -309,7 +351,7 @@ void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(SubqueriesProje subquery_projections.emplace_back(std::pair(select_projection_columns[i], qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName())); - all_subquery_projection_columns.insert(std::pair(select_without_union_query.get(), subquery_projections)); + projection_columns.insert(std::pair(static_cast(select.get()), subquery_projections)); } } } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index 65148e0682a..93e666dde32 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -1,26 +1,14 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include -#include -#include #include namespace DB { -using PredicateExpressions = std::vector; -using ProjectionWithAlias = std::pair; -using ProjectionsWithAliases = std::vector; -using SubqueriesProjectionColumns = std::map; -using IdentifierWithQualifiedName = std::pair; -using IdentifiersWithQualifiedNameSet = std::vector; - +class ASTIdentifier; +class ASTSelectQuery; +class ASTSubquery; +class Context; /** This class provides functions for Push-Down predicate expressions * @@ -35,6 +23,10 @@ using IdentifiersWithQualifiedNameSet = std::vector */ class PredicateExpressionsOptimizer { + using ProjectionWithAlias = std::pair; + using SubqueriesProjectionColumns = std::map>; + using IdentifierWithQualifier = std::pair; + /// Extracts settings, mostly to show which are used and which are not. struct ExtractedSettings { @@ -73,29 +65,29 @@ private: PUSH_TO_HAVING, }; - bool isAggregateFunction(ASTPtr & node); + bool isAggregateFunction(const ASTPtr & node); bool isArrayJoinFunction(const ASTPtr & node); - PredicateExpressions splitConjunctionPredicate(ASTPtr & predicate_expression); + std::vector splitConjunctionPredicate(ASTPtr & predicate_expression); - void getDependenciesAndQualifiedOfExpression(const ASTPtr & expression, IdentifiersWithQualifiedNameSet & dependencies_and_qualified, - std::vector & tables_with_aliases); + std::vector getDependenciesAndQualifiers(ASTPtr & expression, + std::vector & tables_with_aliases); bool optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery); - bool optimizeImpl(ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, bool is_prewhere); + bool optimizeImpl(ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind); - bool cannotPushDownOuterPredicate(const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery, - IdentifiersWithQualifiedNameSet & outer_predicate_dependencies, bool & is_prewhere, OptimizeKind & optimize_kind); + bool canPushDownOuterPredicate(const std::vector & subquery_projection_columns, + const std::vector & outer_predicate_dependencies, + OptimizeKind & optimize_kind); - void cloneOuterPredicateForInnerPredicate(const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns, - std::vector & tables, ASTPtr & inner_predicate); + void setNewAliasesForInnerPredicate(const std::vector & projection_columns, + const std::vector & inner_predicate_dependencies); - void getAllSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns); + SubqueriesProjectionColumns getAllSubqueryProjectionColumns(); - void getSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns, - String & qualified_name_prefix, const ASTPtr & subquery); + void getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & all_subquery_projection_columns); ASTs getSelectQueryProjectionColumns(ASTPtr & ast); diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index a5151be7c64..56529ae595c 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -107,7 +108,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) auto & current_asts = data.current_asts; String & current_alias = data.current_alias; - if (!getColumnIdentifierName(node)) + if (!IdentifierSemantic::getColumnName(node)) return; /// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column"). @@ -124,7 +125,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) if (!my_alias.empty() && my_alias != alias_node->getAliasOrColumnName()) { /// Avoid infinite recursion here - auto opt_name = getColumnIdentifierName(alias_node); + auto opt_name = IdentifierSemantic::getColumnName(alias_node); bool is_cycle = opt_name && *opt_name == node.name; if (!is_cycle) @@ -273,8 +274,7 @@ void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data) visit(child, data); } } - else if (!typeid_cast(node.get()) && - !typeid_cast(node.get())) + else if (!typeid_cast(node.get())) { for (auto & child : node->children) { diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 5dc479fee5e..436e636bec8 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -46,8 +46,7 @@ bool RequiredSourceColumnsMatcher::needChildVisit(ASTPtr & node, const ASTPtr & return false; /// Processed. Do not need children. - if (typeid_cast(node.get()) || - typeid_cast(node.get()) || + if (typeid_cast(node.get()) || typeid_cast(node.get()) || typeid_cast(node.get())) return false; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index fb6a34d37b9..cc9fe20a69b 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -477,20 +478,18 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS std::function get_table_belonging; get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging { - if (getColumnIdentifierName(ast)) + if (IdentifierSemantic::getColumnName(ast)) { auto * identifier = typeid_cast(ast.get()); - { - auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names); - auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names); + size_t left_match_degree = IdentifierSemantic::canReferColumnToTable(*identifier, left_source_names); + size_t right_match_degree = IdentifierSemantic::canReferColumnToTable(*identifier, right_source_names); + + if (left_match_degree > right_match_degree) + return {identifier, nullptr}; + if (left_match_degree < right_match_degree) + return {nullptr, identifier}; - /// Assume that component from definite table if num_components is greater than for the other table. - if (left_num_components > right_num_components) - return {identifier, nullptr}; - if (left_num_components < right_num_components) - return {nullptr, identifier}; - } return {}; } @@ -516,19 +515,15 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS std::function translate_qualified_names; translate_qualified_names = [&](ASTPtr & ast, const DatabaseAndTableWithAlias & source_names, bool right_table) { - if (getColumnIdentifierName(ast)) + if (IdentifierSemantic::getColumnName(ast)) { - auto * identifier = typeid_cast(ast.get()); + auto * identifier = typeid_cast(ast.get()); - { - auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names); - stripIdentifier(ast, num_components); + size_t match = IdentifierSemantic::canReferColumnToTable(*identifier, source_names); + IdentifierSemantic::setColumnShortName(*identifier, match); - if (right_table && source_columns.count(ast->getColumnName())) - source_names.makeQualifiedName(ast); - - } - return; + if (right_table && source_columns.count(ast->getColumnName())) + IdentifierSemantic::setColumnQualifiedName(*identifier, source_names); } for (auto & child : ast->children) diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index f884028dd1c..382c8043f2d 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -28,8 +29,7 @@ bool TranslateQualifiedNamesMatcher::needChildVisit(ASTPtr & node, const ASTPtr return false; /// Processed nodes. Do not go into children. - if (typeid_cast(node.get()) || - typeid_cast(node.get()) || + if (typeid_cast(node.get()) || typeid_cast(node.get())) return false; @@ -50,38 +50,25 @@ std::vector TranslateQualifiedNamesMatcher::visit(ASTPtr & ast, Data & return {}; } -std::vector TranslateQualifiedNamesMatcher::visit(const ASTIdentifier & identifier, ASTPtr & ast, Data & data) +std::vector TranslateQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr & ast, Data & data) { - const NameSet & source_columns = data.source_columns; - const std::vector & tables = data.tables; - - if (getColumnIdentifierName(identifier)) + if (IdentifierSemantic::getColumnName(identifier)) { - /// Select first table name with max number of qualifiers which can be stripped. - size_t max_num_qualifiers_to_strip = 0; - size_t best_table_pos = 0; + bool best_table_pos = 0; + size_t best_match = 0; + for (size_t i = 0; i < data.tables.size(); ++i) + if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, data.tables[i])) + if (match > best_match) + { + best_match = match; + best_table_pos = i; + } - for (size_t table_pos = 0; table_pos < tables.size(); ++table_pos) - { - const auto & table = tables[table_pos]; - auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(identifier, table); - - if (num_qualifiers_to_strip > max_num_qualifiers_to_strip) - { - max_num_qualifiers_to_strip = num_qualifiers_to_strip; - best_table_pos = table_pos; - } - } - - if (max_num_qualifiers_to_strip) - stripIdentifier(ast, max_num_qualifiers_to_strip); + IdentifierSemantic::setColumnShortName(identifier, best_match); /// In case if column from the joined table are in source columns, change it's name to qualified. - if (best_table_pos && source_columns.count(ast->getColumnName())) - { - const DatabaseAndTableWithAlias & table = tables[best_table_pos]; - table.makeQualifiedName(ast); - } + if (best_table_pos && data.source_columns.count(ast->getColumnName())) + IdentifierSemantic::setColumnQualifiedName(identifier, data.tables[best_table_pos]); } return {}; diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h index 2b996bd0370..48d41213cb8 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -30,7 +30,7 @@ public: static bool needChildVisit(ASTPtr & node, const ASTPtr & child); private: - static std::vector visit(const ASTIdentifier & node, ASTPtr & ast, Data &); + static std::vector visit(ASTIdentifier & node, ASTPtr & ast, Data &); static std::vector visit(const ASTQualifiedAsterisk & node, const ASTPtr & ast, Data &); static std::vector visit(ASTTableJoin & node, const ASTPtr & ast, Data &); static std::vector visit(ASTSelectQuery & node, const ASTPtr & ast, Data &); diff --git a/dbms/src/Parsers/ASTIdentifier.cpp b/dbms/src/Parsers/ASTIdentifier.cpp index 81e6bda9741..e5500a89bd0 100644 --- a/dbms/src/Parsers/ASTIdentifier.cpp +++ b/dbms/src/Parsers/ASTIdentifier.cpp @@ -1,12 +1,27 @@ -#include #include +#include #include #include +#include namespace DB { +std::shared_ptr ASTIdentifier::createSpecial(const String & name, std::vector && name_parts) +{ + auto ret = std::make_shared(name, std::move(name_parts)); + ret->semantic->special = true; + return ret; +} + +ASTIdentifier::ASTIdentifier(const String & name_, std::vector && name_parts_) + : name(name_) + , name_parts(name_parts_) + , semantic(std::make_shared()) +{ +} + void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const { auto format_element = [&](const String & elem_name) @@ -74,74 +89,11 @@ bool getIdentifierName(const ASTPtr & ast, String & name) return false; } -std::optional getColumnIdentifierName(const ASTIdentifier & node) -{ - if (!node.special) - return node.name; - return {}; -} - -std::optional getColumnIdentifierName(const ASTPtr & ast) -{ - if (ast) - if (auto id = typeid_cast(ast.get())) - if (!id->special) - return id->name; - return {}; -} - -std::optional getTableIdentifierName(const ASTIdentifier & node) -{ - if (node.special) - return node.name; - return {}; -} - -std::optional getTableIdentifierName(const ASTPtr & ast) -{ - if (ast) - if (auto id = typeid_cast(ast.get())) - if (id->special) - return id->name; - return {}; -} - void setIdentifierSpecial(ASTPtr & ast) { if (ast) if (ASTIdentifier * id = typeid_cast(ast.get())) - id->setSpecial(); -} - -void addIdentifierQualifier(ASTIdentifier & identifier, const String & database, const String & table, const String & alias) -{ - if (!alias.empty()) - { - identifier.name_parts.emplace_back(alias); - } - else - { - if (!database.empty()) - identifier.name_parts.emplace_back(database); - identifier.name_parts.emplace_back(table); - } -} - -bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table) -{ - size_t num_components = identifier.name_parts.size(); - if (num_components >= 3) - return identifier.name_parts[0] == database && - identifier.name_parts[1] == table; - return false; -} - -bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table) -{ - size_t num_components = identifier.name_parts.size(); - if (num_components >= 2) - return identifier.name_parts[0] == table; - return false; + id->semantic->special = true; } } diff --git a/dbms/src/Parsers/ASTIdentifier.h b/dbms/src/Parsers/ASTIdentifier.h index 3b7550f8d5c..5c287eb9da4 100644 --- a/dbms/src/Parsers/ASTIdentifier.h +++ b/dbms/src/Parsers/ASTIdentifier.h @@ -8,6 +8,11 @@ namespace DB { +struct IdentifierSemantic; +struct IdentifierSemanticImpl; +struct DatabaseAndTableWithAlias; + + /// Identifier (column, table or alias) class ASTIdentifier : public ASTWithAlias { @@ -15,14 +20,8 @@ public: /// The composite identifier will have a concatenated name (of the form a.b.c), /// and individual components will be available inside the name_parts. String name; - std::vector name_parts; - ASTIdentifier(const String & name_, std::vector && name_parts_ = {}) - : name(name_) - , name_parts(name_parts_) - , special(false) - { - } + ASTIdentifier(const String & name_, std::vector && name_parts_ = {}); /** Get the text that identifies this element. */ String getID(char delim) const override { return "Identifier" + (delim + name); } @@ -34,6 +33,15 @@ public: set.insert(name); } + bool compound() const { return !name_parts.empty(); } + bool isShort() const { return name_parts.empty() || name == name_parts.back(); } + + void resetWithAlias(const String & new_name) + { + name = new_name; + name_parts.clear(); + } + protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; @@ -41,29 +49,21 @@ protected: private: using ASTWithAlias::children; /// ASTIdentifier is child free - bool special; /// TODO: it would be ptr to semantic here + std::vector name_parts; + std::shared_ptr semantic; /// pimpl - static std::shared_ptr createSpecial(const String & name, std::vector && name_parts = {}) - { - auto ret = std::make_shared(name, std::move(name_parts)); - ret->special = true; - return ret; - } + static std::shared_ptr createSpecial(const String & name, std::vector && name_parts = {}); - void setSpecial() { special = true; } - - friend void setIdentifierSpecial(ASTPtr &); - friend std::optional getColumnIdentifierName(const ASTIdentifier & node); - friend std::optional getColumnIdentifierName(const ASTPtr & ast); - friend std::optional getTableIdentifierName(const ASTIdentifier & node); - friend std::optional getTableIdentifierName(const ASTPtr & ast); + friend struct IdentifierSemantic; friend ASTPtr createTableIdentifier(const String & database_name, const String & table_name); + friend void setIdentifierSpecial(ASTPtr & ast); }; /// ASTIdentifier Helpers: hide casts and semantic. ASTPtr createTableIdentifier(const String & database_name, const String & table_name); +void setIdentifierSpecial(ASTPtr & ast); bool isIdentifier(const IAST * const ast); inline bool isIdentifier(const ASTPtr & ast) { return isIdentifier(ast.get()); } @@ -72,17 +72,5 @@ std::optional getIdentifierName(const IAST * const ast); inline std::optional getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); } bool getIdentifierName(const ASTPtr & ast, String & name); -/// @returns name for column identifiers -std::optional getColumnIdentifierName(const ASTIdentifier & node); -std::optional getColumnIdentifierName(const ASTPtr & ast); - -/// @returns name for 'not a column' identifiers -std::optional getTableIdentifierName(const ASTIdentifier & node); -std::optional getTableIdentifierName(const ASTPtr & ast); - -void setIdentifierSpecial(ASTPtr & ast); -void addIdentifierQualifier(ASTIdentifier & identifier, const String & database, const String & table, const String & alias); -bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table_or_alias); -bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 66a9a6f90d2..d52a855e3f3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -380,7 +381,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr) const if ("indexHint" == function_ptr->name) return true; } - else if (auto opt_name = getColumnIdentifierName(ptr)) + else if (auto opt_name = IdentifierSemantic::getColumnName(ptr)) { /// disallow moving result of ARRAY JOIN to PREWHERE if (array_joined_names.count(*opt_name) || diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index bb31bd81e53..97c085d16e3 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include diff --git a/dbms/src/Storages/VirtualColumnUtils.cpp b/dbms/src/Storages/VirtualColumnUtils.cpp index 990a587445c..b7ea5e66c37 100644 --- a/dbms/src/Storages/VirtualColumnUtils.cpp +++ b/dbms/src/Storages/VirtualColumnUtils.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -96,7 +97,7 @@ static bool isValidFunction(const ASTPtr & expression, const NameSet & columns) if (!isValidFunction(expression->children[i], columns)) return false; - if (auto opt_name = getColumnIdentifierName(expression)) + if (auto opt_name = IdentifierSemantic::getColumnName(expression)) return columns.count(*opt_name); return true; diff --git a/dbms/src/TableFunctions/TableFunctionRemote.cpp b/dbms/src/TableFunctions/TableFunctionRemote.cpp index a41f1c89c49..0d457a01be3 100644 --- a/dbms/src/TableFunctions/TableFunctionRemote.cpp +++ b/dbms/src/TableFunctions/TableFunctionRemote.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index df5aebabc89..bc28d4efb7e 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -13,6 +13,7 @@ 3 3 2000-01-01 1 test string 1 1 3 3 +-------Force push down------- 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql index 2f1af7fa1fa..0180fcdeb1e 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -33,6 +33,7 @@ SELECT * FROM (SELECT toUInt64(b), sum(id) AS b FROM test.test) WHERE `toUInt64( SELECT date, id, name, value FROM (SELECT date, name, value, min(id) AS id FROM test.test GROUP BY date, name, value) WHERE id = 1; SELECT * FROM (SELECT toUInt64(table_alias.b) AS a, sum(id) AS b FROM test.test AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3; +SELECT '-------Force push down-------'; SET force_primary_key = 1; -- Optimize predicate expression with asterisk @@ -65,7 +66,7 @@ SELECT * FROM (SELECT 1 AS id, toDate('2000-01-01') AS date FROM system.numbers SELECT * FROM test.test_view WHERE id = 1; SELECT * FROM test.test_view WHERE id = 2; SELECT id FROM test.test_view WHERE id = 1; -SELECT s.id FROM test.test_view AS s WHERE id = 1; +SELECT s.id FROM test.test_view AS s WHERE s.id = 1; SELECT '-------Push to having expression, need check.-------'; SELECT id FROM (SELECT min(id) AS id FROM test.test) WHERE id = 1; -- { serverError 277 } diff --git a/dbms/tests/queries/0_stateless/00674_join_on_syntax.reference b/dbms/tests/queries/0_stateless/00674_join_on_syntax.reference index ae0944580e5..10c31e56fe8 100644 --- a/dbms/tests/queries/0_stateless/00674_join_on_syntax.reference +++ b/dbms/tests/queries/0_stateless/00674_join_on_syntax.reference @@ -46,8 +46,8 @@ join on complex expression 2 3 2 3 duplicate column names -{"a1":1,"test.tab1_copy.a1":2} -{"a1":1,"test.tab1_copy.a1":2} +{"a1":1,"tab1_copy.a1":2} +{"a1":1,"tab1_copy.a1":2} {"a1":1,"copy.a1":2} {"a1":1,"copy.a1":2} {"a1":1,"copy.a1":2} diff --git a/dbms/tests/queries/0_stateless/00674_join_on_syntax.sql b/dbms/tests/queries/0_stateless/00674_join_on_syntax.sql index 06d930d8c6d..27d83e4684c 100644 --- a/dbms/tests/queries/0_stateless/00674_join_on_syntax.sql +++ b/dbms/tests/queries/0_stateless/00674_join_on_syntax.sql @@ -76,11 +76,11 @@ select a2, b2 from test.tab2 second any left join test.tab3 third on third.a3 + select a2, b2 from test.tab2 second any left join test.tab3 third on third.a3 + test.tab3.b3 = test.tab2.a2 + second.b2; select 'duplicate column names'; -select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; -select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; -select a1, copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; -select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; -select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; +select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; +select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; +select a1, copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; +select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; +select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; select 'subquery'; select a1 from test.tab1 any left join (select * from test.tab2) on b1 = a2; @@ -104,4 +104,4 @@ select a1, a2, b1, b2 from test.tab1 first any left join (select * from test.tab select a1, a2, b1, b2 from test.tab1 first any left join (select *, a2 as z from test.tab2) second on first.b1 = second.z; select a1, a2, b1, b2 from test.tab1 first any left join (select *, a2 + 1 as z from test.tab2) second on first.b1 + 1 = second.z; select tab1.a1, a2, test.tab1.b1, second.b2 from test.tab1 first any left join (select * from test.tab2) second on first.b1 = second.a2; -select a1, s.a1 from test.tab1 any left join (select * from test.tab1_copy) s on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; +select a1, s.a1 from test.tab1 any left join (select * from test.tab1_copy) s on tab1.b1 + 3 = s.b1 + 2 FORMAT JSONEachRow; diff --git a/dbms/tests/queries/0_stateless/00703_join_crash.sql b/dbms/tests/queries/0_stateless/00703_join_crash.sql index 32c0668b3bf..29b86e055fb 100644 --- a/dbms/tests/queries/0_stateless/00703_join_crash.sql +++ b/dbms/tests/queries/0_stateless/00703_join_crash.sql @@ -7,7 +7,7 @@ create table test.tab1_copy (a1 Int32, b1 Int32) engine = MergeTree order by a1; insert into test.tab1 values (1, 2); insert into test.tab1_copy values (2, 3); -select tab1.a1, tab1_copy.a1, tab1.b1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = b1 + 2; +select tab1.a1, tab1_copy.a1, tab1.b1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = tab1_copy.b1 + 2; drop table test.tab1; drop table test.tab1_copy; From 9539467b59ea344971a0bbe2d9ac7d4f108c94ae Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 25 Jan 2019 18:48:53 +0300 Subject: [PATCH 070/324] hotfix --- dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index 382c8043f2d..016d176caba 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -54,7 +54,7 @@ std::vector TranslateQualifiedNamesMatcher::visit(ASTIdentifier & iden { if (IdentifierSemantic::getColumnName(identifier)) { - bool best_table_pos = 0; + size_t best_table_pos = 0; size_t best_match = 0; for (size_t i = 0; i < data.tables.size(); ++i) if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, data.tables[i])) From 0d4b7ff82eac705b182906c66bc41ef81b80b406 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Jan 2019 21:35:16 +0300 Subject: [PATCH 071/324] Refactoring in performance test (may be build, but doesn't work) --- dbms/programs/performance-test/CMakeLists.txt | 6 + .../performance-test/ConfigPreprocessor.cpp | 85 ++ .../performance-test/ConfigPreprocessor.h | 50 + .../performance-test/PerformanceTest.cpp | 1201 ++--------------- .../performance-test/PerformanceTest.h | 49 + .../performance-test/PerformanceTestInfo.cpp | 271 ++++ .../performance-test/PerformanceTestInfo.h | 52 + .../performance-test/PerformanceTestSuite.cpp | 400 ++++++ .../performance-test/ReportBuilder.cpp | 190 +++ .../programs/performance-test/ReportBuilder.h | 30 + dbms/programs/performance-test/TestStats.cpp | 1 + dbms/programs/performance-test/TestStats.h | 2 + .../performance-test/applySubstitutions.cpp | 82 ++ .../performance-test/applySubstitutions.h | 18 + .../performance-test/executeQuery.cpp | 72 + dbms/programs/performance-test/executeQuery.h | 16 + 16 files changed, 1465 insertions(+), 1060 deletions(-) create mode 100644 dbms/programs/performance-test/ConfigPreprocessor.cpp create mode 100644 dbms/programs/performance-test/ConfigPreprocessor.h create mode 100644 dbms/programs/performance-test/PerformanceTest.h create mode 100644 dbms/programs/performance-test/PerformanceTestInfo.cpp create mode 100644 dbms/programs/performance-test/PerformanceTestInfo.h create mode 100644 dbms/programs/performance-test/PerformanceTestSuite.cpp create mode 100644 dbms/programs/performance-test/ReportBuilder.cpp create mode 100644 dbms/programs/performance-test/ReportBuilder.h create mode 100644 dbms/programs/performance-test/applySubstitutions.cpp create mode 100644 dbms/programs/performance-test/applySubstitutions.h create mode 100644 dbms/programs/performance-test/executeQuery.cpp create mode 100644 dbms/programs/performance-test/executeQuery.h diff --git a/dbms/programs/performance-test/CMakeLists.txt b/dbms/programs/performance-test/CMakeLists.txt index 591a7180691..9c1e5e98423 100644 --- a/dbms/programs/performance-test/CMakeLists.txt +++ b/dbms/programs/performance-test/CMakeLists.txt @@ -3,7 +3,13 @@ add_library (clickhouse-performance-test-lib ${LINK_MODE} StopConditionsSet.cpp TestStopConditions.cpp TestStats.cpp + ConfigPreprocessor.cpp PerformanceTest.cpp + PerformanceTestInfo.cpp + executeQuery.cpp + applySubstitutions.cpp + ReportBuilder.cpp + PerformanceTestSuite.cpp ) target_link_libraries (clickhouse-performance-test-lib PRIVATE dbms clickhouse_common_io clickhouse_common_config ${Boost_PROGRAM_OPTIONS_LIBRARY}) target_include_directories (clickhouse-performance-test-lib SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) diff --git a/dbms/programs/performance-test/ConfigPreprocessor.cpp b/dbms/programs/performance-test/ConfigPreprocessor.cpp new file mode 100644 index 00000000000..f03f6d7940f --- /dev/null +++ b/dbms/programs/performance-test/ConfigPreprocessor.cpp @@ -0,0 +1,85 @@ +#include "ConfigPreprocessor.h" +#include +#include +namespace DB +{ +std::vector ConfigPreprocessor::processConfig( + const Strings & tests_tags, + const Strings & tests_names, + const Strings & tests_names_regexp, + const Strings & skip_tags, + const Strings & skip_names, + const Strings & skip_names_regexp) const +{ + + std::vector result; + for (const auto & path : paths) + result.emplace_back(new XMLConfiguration(path)); + /// Leave tests: + removeConfigurationsIf(result, FilterType::Tag, tests_tags, true); + removeConfigurationsIf(result, FilterType::Name, tests_names, true); + removeConfigurationsIf(result, FilterType::Name_regexp, tests_names_regexp, true); + + /// Skip tests + removeConfigurationsIf(result, FilterType::Tag, skip_tags, false); + removeConfigurationsIf(result, FilterType::Name, skip_names, false); + removeConfigurationsIf(result, FilterType::Name_regexp, skip_names_regexp, false); + return result; +} + +void ConfigPreprocessor::removeConfigurationsIf( + std::vector & configs, + ConfigPreprocessor::FilterType filter_type, + const Strings & values, + bool leave) const +{ + auto checker = [&filter_type, &values, &leave] (XMLConfigurationPtr & config) + { + if (values.size() == 0) + return false; + + bool remove_or_not = false; + + if (filter_type == FilterType::Tag) + { + std::vector tags_keys; + config->keys("tags", tags_keys); + + Strings tags(tags_keys.size()); + for (size_t i = 0; i != tags_keys.size(); ++i) + tags[i] = config->getString("tags.tag[" + std::to_string(i) + "]"); + + for (const String & config_tag : tags) + { + if (std::find(values.begin(), values.end(), config_tag) != values.end()) + remove_or_not = true; + } + } + + if (filter_type == FilterType::Name) + { + remove_or_not = (std::find(values.begin(), values.end(), config->getString("name", "")) != values.end()); + } + + if (filter_type == FilterType::Name_regexp) + { + String config_name = config->getString("name", ""); + auto regex_checker = [&config_name](const String & name_regexp) + { + std::regex pattern(name_regexp); + return std::regex_search(config_name, pattern); + }; + + remove_or_not = config->has("name") ? (std::find_if(values.begin(), values.end(), regex_checker) != values.end()) : false; + } + + if (leave) + remove_or_not = !remove_or_not; + return remove_or_not; + }; + + auto new_end = std::remove_if(configs.begin(), configs.end(), checker); + configs.erase(new_end, configs.end()); +} + +} diff --git a/dbms/programs/performance-test/ConfigPreprocessor.h b/dbms/programs/performance-test/ConfigPreprocessor.h new file mode 100644 index 00000000000..49c85032b93 --- /dev/null +++ b/dbms/programs/performance-test/ConfigPreprocessor.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +using XMLConfiguration = Poco::Util::XMLConfiguration; +using XMLConfigurationPtr = Poco::AutoPtr; +using XMLDocumentPtr = Poco::AutoPtr; +using Strings = std::vector; + +class ConfigPreprocessor +{ +public: + ConfigPreprocessor(const std::vector & paths_) + : paths(paths_) + {} + + std::vector processConfig( + const Strings & tests_tags, + const Strings & tests_names, + const Strings & tests_names_regexp, + const Strings & skip_tags, + const Strings & skip_names, + const Strings & skip_names_regexp) const; + +private: + + enum class FilterType + { + Tag, + Name, + Name_regexp + }; + + /// Removes configurations that has a given value. + /// If leave is true, the logic is reversed. + void removeConfigurationsIf( + std::vector & configs, + FilterType filter_type, + const Strings & values, + bool leave = false) const; + + const std::vector paths; +}; +} diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index d5bfcc85c60..88b9617013c 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -1,1097 +1,178 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include "PerformanceTest.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include -#include -#include -#include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "JSONString.h" -#include "StopConditionsSet.h" -#include "TestStopConditions.h" -#include "TestStats.h" - -#ifndef __clang__ -#pragma GCC optimize("-fno-var-tracking-assignments") -#endif - - -/** Tests launcher for ClickHouse. - * The tool walks through given or default folder in order to find files with - * tests' descriptions and launches it. - */ -namespace fs = boost::filesystem; -using String = std::string; -const std::regex QUOTE_REGEX{"\""}; +#include +#include +#include "executeQuery.h" namespace DB { + namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; - extern const int FILE_DOESNT_EXIST; +extern const int NOT_IMPLEMENTED; +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +extern const int FILE_DOESNT_EXIST; +} + +namespace fs = boost::filesystem; + +PerformanceTest::PerformanceTest( + const XMLConfigurationPtr & config_, + Connection & connection_, + InterruptListener & interrupt_listener_, + const PerformanceTestInfo & test_info_) + : config(config_) + , connection(connection_) + , interrupt_listener(interrupt_listener_) + , test_info(test_info_) +{ +} + +bool PerformanceTest::checkPreconditions() const +{ + if (!config->has("preconditions")) + return true; + + std::vector preconditions; + config->keys("preconditions", preconditions); + size_t table_precondition_index = 0; + + for (const String & precondition : preconditions) + { + if (precondition == "flush_disk_cache") + { + if (system( + "(>&2 echo 'Flushing disk cache...') && (sudo sh -c 'echo 3 > /proc/sys/vm/drop_caches') && (>&2 echo 'Flushed.')")) + { + std::cerr << "Failed to flush disk cache" << std::endl; + return false; + } + } + + if (precondition == "ram_size") + { + size_t ram_size_needed = config->getUInt64("preconditions.ram_size"); + size_t actual_ram = getMemoryAmount(); + if (!actual_ram) + throw DB::Exception("ram_size precondition not available on this platform", DB::ErrorCodes::NOT_IMPLEMENTED); + + if (ram_size_needed > actual_ram) + { + std::cerr << "Not enough RAM: need = " << ram_size_needed << ", present = " << actual_ram << std::endl; + return false; + } + } + + if (precondition == "table_exists") + { + String precondition_key = "preconditions.table_exists[" + std::to_string(table_precondition_index++) + "]"; + String table_to_check = config->getString(precondition_key); + String query = "EXISTS TABLE " + table_to_check + ";"; + + size_t exist = 0; + + connection.sendQuery(query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false); + + while (true) + { + Connection::Packet packet = connection.receivePacket(); + + if (packet.type == Protocol::Server::Data) + { + for (const ColumnWithTypeAndName & column : packet.block) + { + if (column.name == "result" && column.column->size() > 0) + { + exist = column.column->get64(0); + if (exist) + break; + } + } + } + + if (packet.type == Protocol::Server::Exception + || packet.type == Protocol::Server::EndOfStream) + break; + } + + if (!exist) + { + std::cerr << "Table " << table_to_check << " doesn't exist" << std::endl; + return false; + } + } + } + + return true; } -using ConfigurationPtr = Poco::AutoPtr; -class PerformanceTest : public Poco::Util::Application +std::vector PerformanceTest::execute() { -public: - using Strings = std::vector; - - PerformanceTest(const String & host_, - const UInt16 port_, - const bool secure_, - const String & default_database_, - const String & user_, - const String & password_, - const bool lite_output_, - const String & profiles_file_, - Strings && input_files_, - Strings && tests_tags_, - Strings && skip_tags_, - Strings && tests_names_, - Strings && skip_names_, - Strings && tests_names_regexp_, - Strings && skip_names_regexp_, - const ConnectionTimeouts & timeouts) - : connection(host_, port_, default_database_, user_, password_, timeouts, "performance-test", Protocol::Compression::Enable, secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable), - gotSIGINT(false), - lite_output(lite_output_), - profiles_file(profiles_file_), - input_files(input_files_), - tests_tags(std::move(tests_tags_)), - skip_tags(std::move(skip_tags_)), - tests_names(std::move(tests_names_)), - skip_names(std::move(skip_names_)), - tests_names_regexp(std::move(tests_names_regexp_)), - skip_names_regexp(std::move(skip_names_regexp_)) - { - if (input_files.size() < 1) - { - throw DB::Exception("No tests were specified", DB::ErrorCodes::BAD_ARGUMENTS); - } - } - - void initialize(Poco::Util::Application & self [[maybe_unused]]) - { - std::string home_path; - const char * home_path_cstr = getenv("HOME"); - if (home_path_cstr) - home_path = home_path_cstr; - configReadClient(Poco::Util::Application::instance().config(), home_path); - } - - int main(const std::vector < std::string > & /* args */) - { - std::string name; - UInt64 version_major; - UInt64 version_minor; - UInt64 version_patch; - UInt64 version_revision; - connection.getServerVersion(name, version_major, version_minor, version_patch, version_revision); - - std::stringstream ss; - ss << version_major << "." << version_minor << "." << version_patch; - server_version = ss.str(); - - processTestsConfigurations(input_files); - - return 0; - } - -private: - String test_name; - - using Query = String; - using Queries = std::vector; - using QueriesWithIndexes = std::vector>; - Queries queries; - - Connection connection; - std::string server_version; - - using Keys = std::vector; - - Settings settings; - Context global_context = Context::createGlobal(); - - InterruptListener interrupt_listener; - - using XMLConfiguration = Poco::Util::XMLConfiguration; - using XMLConfigurationPtr = Poco::AutoPtr; - - using Paths = std::vector; - using StringToVector = std::map>; - using StringToMap = std::map; - StringToMap substitutions; - - using StringKeyValue = std::map; - std::vector substitutions_maps; - - bool gotSIGINT; - std::vector stop_conditions_by_run; - String main_metric; - bool lite_output; - String profiles_file; - - Strings input_files; - std::vector tests_configurations; - - Strings tests_tags; - Strings skip_tags; - Strings tests_names; - Strings skip_names; - Strings tests_names_regexp; - Strings skip_names_regexp; - - enum class ExecutionType - { - Loop, - Once - }; - ExecutionType exec_type; - - enum class FilterType - { - Tag, - Name, - Name_regexp - }; - - size_t times_to_run = 1; std::vector statistics_by_run; - - /// Removes configurations that has a given value. If leave is true, the logic is reversed. - void removeConfigurationsIf( - std::vector & configs, FilterType filter_type, const Strings & values, bool leave = false) + statistics_by_run.resize(test_info.times_to_run * test_info.queries.size()); + for (size_t number_of_launch = 0; number_of_launch < test_info.times_to_run; ++number_of_launch) { - auto checker = [&filter_type, &values, &leave](XMLConfigurationPtr & config) + QueriesWithIndexes queries_with_indexes; + + for (size_t query_index = 0; query_index < test_info.queries.size(); ++query_index) { - if (values.size() == 0) - return false; + size_t statistic_index = number_of_launch * test_info.queries.size() + query_index; + test_info.stop_conditions_by_run[statistic_index].reset(); - bool remove_or_not = false; - - if (filter_type == FilterType::Tag) - { - Keys tags_keys; - config->keys("tags", tags_keys); - - Strings tags(tags_keys.size()); - for (size_t i = 0; i != tags_keys.size(); ++i) - tags[i] = config->getString("tags.tag[" + std::to_string(i) + "]"); - - for (const String & config_tag : tags) - { - if (std::find(values.begin(), values.end(), config_tag) != values.end()) - remove_or_not = true; - } - } - - if (filter_type == FilterType::Name) - { - remove_or_not = (std::find(values.begin(), values.end(), config->getString("name", "")) != values.end()); - } - - if (filter_type == FilterType::Name_regexp) - { - String config_name = config->getString("name", ""); - auto regex_checker = [&config_name](const String & name_regexp) - { - std::regex pattern(name_regexp); - return std::regex_search(config_name, pattern); - }; - - remove_or_not = config->has("name") ? (std::find_if(values.begin(), values.end(), regex_checker) != values.end()) : false; - } - - if (leave) - remove_or_not = !remove_or_not; - return remove_or_not; - }; - - auto new_end = std::remove_if(configs.begin(), configs.end(), checker); - configs.erase(new_end, configs.end()); - } - - /// Filter tests by tags, names, regexp matching, etc. - void filterConfigurations() - { - /// Leave tests: - removeConfigurationsIf(tests_configurations, FilterType::Tag, tests_tags, true); - removeConfigurationsIf(tests_configurations, FilterType::Name, tests_names, true); - removeConfigurationsIf(tests_configurations, FilterType::Name_regexp, tests_names_regexp, true); - - - /// Skip tests - removeConfigurationsIf(tests_configurations, FilterType::Tag, skip_tags, false); - removeConfigurationsIf(tests_configurations, FilterType::Name, skip_names, false); - removeConfigurationsIf(tests_configurations, FilterType::Name_regexp, skip_names_regexp, false); - } - - /// Checks specified preconditions per test (process cache, table existence, etc.) - bool checkPreconditions(const XMLConfigurationPtr & config) - { - if (!config->has("preconditions")) - return true; - - Keys preconditions; - config->keys("preconditions", preconditions); - size_t table_precondition_index = 0; - - for (const String & precondition : preconditions) - { - if (precondition == "flush_disk_cache") - { - if (system( - "(>&2 echo 'Flushing disk cache...') && (sudo sh -c 'echo 3 > /proc/sys/vm/drop_caches') && (>&2 echo 'Flushed.')")) - { - std::cerr << "Failed to flush disk cache" << std::endl; - return false; - } - } - - if (precondition == "ram_size") - { - size_t ram_size_needed = config->getUInt64("preconditions.ram_size"); - size_t actual_ram = getMemoryAmount(); - if (!actual_ram) - throw DB::Exception("ram_size precondition not available on this platform", DB::ErrorCodes::NOT_IMPLEMENTED); - - if (ram_size_needed > actual_ram) - { - std::cerr << "Not enough RAM: need = " << ram_size_needed << ", present = " << actual_ram << std::endl; - return false; - } - } - - if (precondition == "table_exists") - { - String precondition_key = "preconditions.table_exists[" + std::to_string(table_precondition_index++) + "]"; - String table_to_check = config->getString(precondition_key); - String query = "EXISTS TABLE " + table_to_check + ";"; - - size_t exist = 0; - - connection.sendQuery(query, "", QueryProcessingStage::Complete, &settings, nullptr, false); - - while (true) - { - Connection::Packet packet = connection.receivePacket(); - - if (packet.type == Protocol::Server::Data) - { - for (const ColumnWithTypeAndName & column : packet.block) - { - if (column.name == "result" && column.column->size() > 0) - { - exist = column.column->get64(0); - if (exist) - break; - } - } - } - - if (packet.type == Protocol::Server::Exception || packet.type == Protocol::Server::EndOfStream) - break; - } - - if (!exist) - { - std::cerr << "Table " << table_to_check << " doesn't exist" << std::endl; - return false; - } - } - } - - return true; - } - - void processTestsConfigurations(const Paths & paths) - { - tests_configurations.resize(paths.size()); - - for (size_t i = 0; i != paths.size(); ++i) - { - const String path = paths[i]; - tests_configurations[i] = XMLConfigurationPtr(new XMLConfiguration(path)); - } - - filterConfigurations(); - - if (tests_configurations.size()) - { - Strings outputs; - - for (auto & test_config : tests_configurations) - { - if (!checkPreconditions(test_config)) - { - std::cerr << "Preconditions are not fulfilled for test '" + test_config->getString("name", "") + "' "; - continue; - } - - String output = runTest(test_config); - if (lite_output) - std::cout << output; - else - outputs.push_back(output); - } - - if (!lite_output && outputs.size()) - { - std::cout << "[" << std::endl; - - for (size_t i = 0; i != outputs.size(); ++i) - { - std::cout << outputs[i]; - if (i != outputs.size() - 1) - std::cout << ","; - - std::cout << std::endl; - } - - std::cout << "]" << std::endl; - } - } - } - - void extractSettings( - const XMLConfigurationPtr & config, const String & key, const Strings & settings_list, std::map & settings_to_apply) - { - for (const String & setup : settings_list) - { - if (setup == "profile") - continue; - - String value = config->getString(key + "." + setup); - if (value.empty()) - value = "true"; - - settings_to_apply[setup] = value; - } - } - - String runTest(XMLConfigurationPtr & test_config) - { - queries.clear(); - - test_name = test_config->getString("name"); - std::cerr << "Running: " << test_name << "\n"; - - if (test_config->has("settings")) - { - std::map settings_to_apply; - Keys config_settings; - test_config->keys("settings", config_settings); - - /// Preprocess configuration file - if (std::find(config_settings.begin(), config_settings.end(), "profile") != config_settings.end()) - { - if (!profiles_file.empty()) - { - String profile_name = test_config->getString("settings.profile"); - XMLConfigurationPtr profiles_config(new XMLConfiguration(profiles_file)); - - Keys profile_settings; - profiles_config->keys("profiles." + profile_name, profile_settings); - - extractSettings(profiles_config, "profiles." + profile_name, profile_settings, settings_to_apply); - } - } - - extractSettings(test_config, "settings", config_settings, settings_to_apply); - - /// This macro goes through all settings in the Settings.h - /// and, if found any settings in test's xml configuration - /// with the same name, sets its value to settings - std::map::iterator it; -#define EXTRACT_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) \ - it = settings_to_apply.find(#NAME); \ - if (it != settings_to_apply.end()) \ - settings.set(#NAME, settings_to_apply[#NAME]); - - APPLY_FOR_SETTINGS(EXTRACT_SETTING) - -#undef EXTRACT_SETTING - - if (std::find(config_settings.begin(), config_settings.end(), "average_rows_speed_precision") != config_settings.end()) - { - TestStats::avg_rows_speed_precision = test_config->getDouble("settings.average_rows_speed_precision"); - } - - if (std::find(config_settings.begin(), config_settings.end(), "average_bytes_speed_precision") != config_settings.end()) - { - TestStats::avg_bytes_speed_precision = test_config->getDouble("settings.average_bytes_speed_precision"); - } - } - - if (!test_config->has("query") && !test_config->has("query_file")) - { - throw DB::Exception("Missing query fields in test's config: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS); - } - - if (test_config->has("query") && test_config->has("query_file")) - { - throw DB::Exception("Found both query and query_file fields. Choose only one", DB::ErrorCodes::BAD_ARGUMENTS); - } - - if (test_config->has("query")) - { - queries = DB::getMultipleValuesFromConfig(*test_config, "", "query"); - } - - if (test_config->has("query_file")) - { - const String filename = test_config->getString("query_file"); - if (filename.empty()) - throw DB::Exception("Empty file name", DB::ErrorCodes::BAD_ARGUMENTS); - - bool tsv = fs::path(filename).extension().string() == ".tsv"; - - ReadBufferFromFile query_file(filename); - Query query; - - if (tsv) - { - while (!query_file.eof()) - { - readEscapedString(query, query_file); - assertChar('\n', query_file); - queries.push_back(query); - } - } - else - { - readStringUntilEOF(query, query_file); - queries.push_back(query); - } - } - - if (queries.empty()) - { - throw DB::Exception("Did not find any query to execute: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS); - } - - if (test_config->has("substitutions")) - { - /// Make "subconfig" of inner xml block - ConfigurationPtr substitutions_view(test_config->createView("substitutions")); - constructSubstitutions(substitutions_view, substitutions[test_name]); - - auto queries_pre_format = queries; - queries.clear(); - for (const auto & query : queries_pre_format) - { - auto formatted = formatQueries(query, substitutions[test_name]); - queries.insert(queries.end(), formatted.begin(), formatted.end()); - } - } - - if (!test_config->has("type")) - { - throw DB::Exception("Missing type property in config: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS); - } - - String config_exec_type = test_config->getString("type"); - if (config_exec_type == "loop") - exec_type = ExecutionType::Loop; - else if (config_exec_type == "once") - exec_type = ExecutionType::Once; - else - throw DB::Exception("Unknown type " + config_exec_type + " in :" + test_name, DB::ErrorCodes::BAD_ARGUMENTS); - - times_to_run = test_config->getUInt("times_to_run", 1); - - stop_conditions_by_run.clear(); - TestStopConditions stop_conditions_template; - if (test_config->has("stop_conditions")) - { - ConfigurationPtr stop_conditions_config(test_config->createView("stop_conditions")); - stop_conditions_template.loadFromConfig(stop_conditions_config); - } - - if (stop_conditions_template.empty()) - throw DB::Exception("No termination conditions were found in config", DB::ErrorCodes::BAD_ARGUMENTS); - - for (size_t i = 0; i < times_to_run * queries.size(); ++i) - stop_conditions_by_run.push_back(stop_conditions_template); - - - ConfigurationPtr metrics_view(test_config->createView("metrics")); - Keys metrics; - metrics_view->keys(metrics); - - main_metric.clear(); - if (test_config->has("main_metric")) - { - Keys main_metrics; - test_config->keys("main_metric", main_metrics); - if (main_metrics.size()) - main_metric = main_metrics[0]; - } - - if (!main_metric.empty()) - { - if (std::find(metrics.begin(), metrics.end(), main_metric) == metrics.end()) - metrics.push_back(main_metric); - } - else - { - if (metrics.empty()) - throw DB::Exception("You shoud specify at least one metric", DB::ErrorCodes::BAD_ARGUMENTS); - main_metric = metrics[0]; - if (lite_output) - throw DB::Exception("Specify main_metric for lite output", DB::ErrorCodes::BAD_ARGUMENTS); - } - - if (metrics.size() > 0) - checkMetricsInput(metrics); - - statistics_by_run.resize(times_to_run * queries.size()); - for (size_t number_of_launch = 0; number_of_launch < times_to_run; ++number_of_launch) - { - QueriesWithIndexes queries_with_indexes; - - for (size_t query_index = 0; query_index < queries.size(); ++query_index) - { - size_t statistic_index = number_of_launch * queries.size() + query_index; - stop_conditions_by_run[statistic_index].reset(); - - queries_with_indexes.push_back({queries[query_index], statistic_index}); - } - - if (interrupt_listener.check()) - gotSIGINT = true; - - if (gotSIGINT) - break; - - runQueries(queries_with_indexes); - } - - if (lite_output) - return minOutput(); - else - return constructTotalInfo(metrics); - } - - void checkMetricsInput(const Strings & metrics) const - { - std::vector loop_metrics - = {"min_time", "quantiles", "total_time", "queries_per_second", "rows_per_second", "bytes_per_second"}; - - std::vector non_loop_metrics - = {"max_rows_per_second", "max_bytes_per_second", "avg_rows_per_second", "avg_bytes_per_second"}; - - if (exec_type == ExecutionType::Loop) - { - for (const String & metric : metrics) - if (std::find(non_loop_metrics.begin(), non_loop_metrics.end(), metric) != non_loop_metrics.end()) - throw DB::Exception("Wrong type of metric for loop execution type (" + metric + ")", DB::ErrorCodes::BAD_ARGUMENTS); - } - else - { - for (const String & metric : metrics) - if (std::find(loop_metrics.begin(), loop_metrics.end(), metric) != loop_metrics.end()) - throw DB::Exception("Wrong type of metric for non-loop execution type (" + metric + ")", DB::ErrorCodes::BAD_ARGUMENTS); - } - } - - void runQueries(const QueriesWithIndexes & queries_with_indexes) - { - for (const auto & [query, run_index] : queries_with_indexes) - { - TestStopConditions & stop_conditions = stop_conditions_by_run[run_index]; - TestStats & statistics = statistics_by_run[run_index]; - - statistics.clear(); - try - { - execute(query, statistics, stop_conditions); - - if (exec_type == ExecutionType::Loop) - { - for (size_t iteration = 1; !gotSIGINT; ++iteration) - { - stop_conditions.reportIterations(iteration); - if (stop_conditions.areFulfilled()) - break; - - execute(query, statistics, stop_conditions); - } - } - } - catch (const DB::Exception & e) - { - statistics.exception = e.what() + String(", ") + e.displayText(); - } - - if (!gotSIGINT) - { - statistics.ready = true; - } - } - } - - void execute(const Query & query, TestStats & statistics, TestStopConditions & stop_conditions) - { - statistics.watch_per_query.restart(); - statistics.last_query_was_cancelled = false; - statistics.last_query_rows_read = 0; - statistics.last_query_bytes_read = 0; - - RemoteBlockInputStream stream(connection, query, {}, global_context, &settings); - - stream.setProgressCallback( - [&](const Progress & value) { this->checkFulfilledConditionsAndUpdate(value, stream, statistics, stop_conditions); }); - - stream.readPrefix(); - while (Block block = stream.read()) - ; - stream.readSuffix(); - - if (!statistics.last_query_was_cancelled) - statistics.updateQueryInfo(); - - statistics.setTotalTime(); - } - - void checkFulfilledConditionsAndUpdate( - const Progress & progress, RemoteBlockInputStream & stream, TestStats & statistics, TestStopConditions & stop_conditions) - { - statistics.add(progress.rows, progress.bytes); - - stop_conditions.reportRowsRead(statistics.total_rows_read); - stop_conditions.reportBytesReadUncompressed(statistics.total_bytes_read); - stop_conditions.reportTotalTime(statistics.watch.elapsed() / (1000 * 1000)); - stop_conditions.reportMinTimeNotChangingFor(statistics.min_time_watch.elapsed() / (1000 * 1000)); - stop_conditions.reportMaxSpeedNotChangingFor(statistics.max_rows_speed_watch.elapsed() / (1000 * 1000)); - stop_conditions.reportAverageSpeedNotChangingFor(statistics.avg_rows_speed_watch.elapsed() / (1000 * 1000)); - - if (stop_conditions.areFulfilled()) - { - statistics.last_query_was_cancelled = true; - stream.cancel(false); + queries_with_indexes.push_back({test_info.queries[query_index], statistic_index}); } if (interrupt_listener.check()) - { - gotSIGINT = true; - statistics.last_query_was_cancelled = true; - stream.cancel(false); - } + break; + + runQueries(queries_with_indexes, statistics_by_run); } - - void constructSubstitutions(ConfigurationPtr & substitutions_view, StringToVector & out_substitutions) - { - Keys xml_substitutions; - substitutions_view->keys(xml_substitutions); - - for (size_t i = 0; i != xml_substitutions.size(); ++i) - { - const ConfigurationPtr xml_substitution(substitutions_view->createView("substitution[" + std::to_string(i) + "]")); - - /// Property values for substitution will be stored in a vector - /// accessible by property name - std::vector xml_values; - xml_substitution->keys("values", xml_values); - - String name = xml_substitution->getString("name"); - - for (size_t j = 0; j != xml_values.size(); ++j) - { - out_substitutions[name].push_back(xml_substitution->getString("values.value[" + std::to_string(j) + "]")); - } - } - } - - std::vector formatQueries(const String & query, StringToVector substitutions_to_generate) - { - std::vector queries_res; - runThroughAllOptionsAndPush(substitutions_to_generate.begin(), substitutions_to_generate.end(), query, queries_res); - return queries_res; - } - - /// Recursive method which goes through all substitution blocks in xml - /// and replaces property {names} by their values - void runThroughAllOptionsAndPush(StringToVector::iterator substitutions_left, - StringToVector::iterator substitutions_right, - const String & template_query, - std::vector & out_queries) - { - if (substitutions_left == substitutions_right) - { - out_queries.push_back(template_query); /// completely substituted query - return; - } - - String substitution_mask = "{" + substitutions_left->first + "}"; - - if (template_query.find(substitution_mask) == String::npos) /// nothing to substitute here - { - runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, template_query, out_queries); - return; - } - - for (const String & value : substitutions_left->second) - { - /// Copy query string for each unique permutation - Query query = template_query; - size_t substr_pos = 0; - - while (substr_pos != String::npos) - { - substr_pos = query.find(substitution_mask); - - if (substr_pos != String::npos) - query.replace(substr_pos, substitution_mask.length(), value); - } - - runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, query, out_queries); - } - } - -public: - String constructTotalInfo(Strings metrics) - { - JSONString json_output; - - json_output.set("hostname", getFQDNOrHostName()); - json_output.set("num_cores", getNumberOfPhysicalCPUCores()); - json_output.set("num_threads", std::thread::hardware_concurrency()); - json_output.set("ram", getMemoryAmount()); - json_output.set("server_version", server_version); - json_output.set("time", DateLUT::instance().timeToString(time(nullptr))); - json_output.set("test_name", test_name); - json_output.set("main_metric", main_metric); - - if (substitutions[test_name].size()) - { - JSONString json_parameters(2); /// here, 2 is the size of \t padding - - for (auto it = substitutions[test_name].begin(); it != substitutions[test_name].end(); ++it) - { - String parameter = it->first; - std::vector values = it->second; - - String array_string = "["; - for (size_t i = 0; i != values.size(); ++i) - { - array_string += '"' + std::regex_replace(values[i], QUOTE_REGEX, "\\\"") + '"'; - if (i != values.size() - 1) - { - array_string += ", "; - } - } - array_string += ']'; - - json_parameters.set(parameter, array_string); - } - - json_output.set("parameters", json_parameters.asString()); - } - - std::vector run_infos; - for (size_t query_index = 0; query_index < queries.size(); ++query_index) - { - for (size_t number_of_launch = 0; number_of_launch < times_to_run; ++number_of_launch) - { - TestStats & statistics = statistics_by_run[number_of_launch * queries.size() + query_index]; - - if (!statistics.ready) - continue; - - JSONString runJSON; - - runJSON.set("query", std::regex_replace(queries[query_index], QUOTE_REGEX, "\\\"")); - if (!statistics.exception.empty()) - runJSON.set("exception", statistics.exception); - - if (substitutions_maps.size()) - { - JSONString parameters(4); - - for (auto it = substitutions_maps[query_index].begin(); it != substitutions_maps[query_index].end(); ++it) - { - parameters.set(it->first, it->second); - } - - runJSON.set("parameters", parameters.asString()); - } - - - if (exec_type == ExecutionType::Loop) - { - /// in seconds - if (std::find(metrics.begin(), metrics.end(), "min_time") != metrics.end()) - runJSON.set("min_time", statistics.min_time / double(1000)); - - if (std::find(metrics.begin(), metrics.end(), "quantiles") != metrics.end()) - { - JSONString quantiles(4); /// here, 4 is the size of \t padding - for (double percent = 10; percent <= 90; percent += 10) - { - String quantile_key = std::to_string(percent / 100.0); - while (quantile_key.back() == '0') - quantile_key.pop_back(); - - quantiles.set(quantile_key, statistics.sampler.quantileInterpolated(percent / 100.0)); - } - quantiles.set("0.95", statistics.sampler.quantileInterpolated(95 / 100.0)); - quantiles.set("0.99", statistics.sampler.quantileInterpolated(99 / 100.0)); - quantiles.set("0.999", statistics.sampler.quantileInterpolated(99.9 / 100.0)); - quantiles.set("0.9999", statistics.sampler.quantileInterpolated(99.99 / 100.0)); - - runJSON.set("quantiles", quantiles.asString()); - } - - if (std::find(metrics.begin(), metrics.end(), "total_time") != metrics.end()) - runJSON.set("total_time", statistics.total_time); - - if (std::find(metrics.begin(), metrics.end(), "queries_per_second") != metrics.end()) - runJSON.set("queries_per_second", double(statistics.queries) / statistics.total_time); - - if (std::find(metrics.begin(), metrics.end(), "rows_per_second") != metrics.end()) - runJSON.set("rows_per_second", double(statistics.total_rows_read) / statistics.total_time); - - if (std::find(metrics.begin(), metrics.end(), "bytes_per_second") != metrics.end()) - runJSON.set("bytes_per_second", double(statistics.total_bytes_read) / statistics.total_time); - } - else - { - if (std::find(metrics.begin(), metrics.end(), "max_rows_per_second") != metrics.end()) - runJSON.set("max_rows_per_second", statistics.max_rows_speed); - - if (std::find(metrics.begin(), metrics.end(), "max_bytes_per_second") != metrics.end()) - runJSON.set("max_bytes_per_second", statistics.max_bytes_speed); - - if (std::find(metrics.begin(), metrics.end(), "avg_rows_per_second") != metrics.end()) - runJSON.set("avg_rows_per_second", statistics.avg_rows_speed_value); - - if (std::find(metrics.begin(), metrics.end(), "avg_bytes_per_second") != metrics.end()) - runJSON.set("avg_bytes_per_second", statistics.avg_bytes_speed_value); - } - - run_infos.push_back(runJSON); - } - } - - json_output.set("runs", run_infos); - - return json_output.asString(); - } - - String minOutput() - { - String output; - - for (size_t query_index = 0; query_index < queries.size(); ++query_index) - { - for (size_t number_of_launch = 0; number_of_launch < times_to_run; ++number_of_launch) - { - if (queries.size() > 1) - { - output += "query \"" + queries[query_index] + "\", "; - } - - if (substitutions_maps.size()) - { - for (auto it = substitutions_maps[query_index].begin(); it != substitutions_maps[query_index].end(); ++it) - { - output += it->first + " = " + it->second + ", "; - } - } - - output += "run " + std::to_string(number_of_launch + 1) + ": "; - output += main_metric + " = "; - output += statistics_by_run[number_of_launch * queries.size() + query_index].getStatisticByName(main_metric); - output += "\n"; - } - } - - return output; - } -}; + return statistics_by_run; } -static void getFilesFromDir(const fs::path & dir, std::vector & input_files, const bool recursive = false) + +void PerformanceTest::runQueries( + const QueriesWithIndexes & queries_with_indexes, + std::vector & statistics_by_run) { - if (dir.extension().string() == ".xml") - std::cerr << "Warning: '" + dir.string() + "' is a directory, but has .xml extension" << std::endl; - - fs::directory_iterator end; - for (fs::directory_iterator it(dir); it != end; ++it) + for (const auto & [query, run_index] : queries_with_indexes) { - const fs::path file = (*it); - if (recursive && fs::is_directory(file)) - getFilesFromDir(file, input_files, recursive); - else if (!fs::is_directory(file) && file.extension().string() == ".xml") - input_files.push_back(file.string()); - } -} + TestStopConditions & stop_conditions = test_info.stop_conditions_by_run[run_index]; + TestStats & statistics = statistics_by_run[run_index]; - -int mainEntryClickHousePerformanceTest(int argc, char ** argv) -try -{ - using boost::program_options::value; - using Strings = std::vector; - - boost::program_options::options_description desc("Allowed options"); - desc.add_options() - ("help", "produce help message") - ("lite", "use lite version of output") - ("profiles-file", value()->default_value(""), "Specify a file with global profiles") - ("host,h", value()->default_value("localhost"), "") - ("port", value()->default_value(9000), "") - ("secure,s", "Use TLS connection") - ("database", value()->default_value("default"), "") - ("user", value()->default_value("default"), "") - ("password", value()->default_value(""), "") - ("tags", value()->multitoken(), "Run only tests with tag") - ("skip-tags", value()->multitoken(), "Do not run tests with tag") - ("names", value()->multitoken(), "Run tests with specific name") - ("skip-names", value()->multitoken(), "Do not run tests with name") - ("names-regexp", value()->multitoken(), "Run tests with names matching regexp") - ("skip-names-regexp", value()->multitoken(), "Do not run tests with names matching regexp") - ("recursive,r", "Recurse in directories to find all xml's"); - - /// These options will not be displayed in --help - boost::program_options::options_description hidden("Hidden options"); - hidden.add_options() - ("input-files", value>(), ""); - - /// But they will be legit, though. And they must be given without name - boost::program_options::positional_options_description positional; - positional.add("input-files", -1); - - boost::program_options::options_description cmdline_options; - cmdline_options.add(desc).add(hidden); - - boost::program_options::variables_map options; - boost::program_options::store( - boost::program_options::command_line_parser(argc, argv).options(cmdline_options).positional(positional).run(), options); - boost::program_options::notify(options); - - if (options.count("help")) - { - std::cout << "Usage: " << argv[0] << " [options] [test_file ...] [tests_folder]\n"; - std::cout << desc << "\n"; - return 0; - } - - Strings input_files; - bool recursive = options.count("recursive"); - - if (!options.count("input-files")) - { - std::cerr << "Trying to find test scenario files in the current folder..."; - fs::path curr_dir("."); - - getFilesFromDir(curr_dir, input_files, recursive); - - if (input_files.empty()) + statistics.clear(); + try { - std::cerr << std::endl; - throw DB::Exception("Did not find any xml files", DB::ErrorCodes::BAD_ARGUMENTS); - } - else - std::cerr << " found " << input_files.size() << " files." << std::endl; - } - else - { - input_files = options["input-files"].as(); - Strings collected_files; + executeQuery(connection, query, statistics, stop_conditions, interrupt_listener); - for (const String & filename : input_files) - { - fs::path file(filename); - - if (!fs::exists(file)) - throw DB::Exception("File '" + filename + "' does not exist", DB::ErrorCodes::FILE_DOESNT_EXIST); - - if (fs::is_directory(file)) + if (test_info.exec_type == ExecutionType::Loop) { - getFilesFromDir(file, collected_files, recursive); - } - else - { - if (file.extension().string() != ".xml") - throw DB::Exception("File '" + filename + "' does not have .xml extension", DB::ErrorCodes::BAD_ARGUMENTS); - collected_files.push_back(filename); + for (size_t iteration = 1; !statistics.got_SIGINT; ++iteration) + { + stop_conditions.reportIterations(iteration); + if (stop_conditions.areFulfilled()) + break; + + executeQuery(connection, query, statistics, stop_conditions, interrupt_listener); + } } } + catch (const DB::Exception & e) + { + statistics.exception = e.what() + String(", ") + e.displayText(); + } - input_files = std::move(collected_files); + if (!statistics.got_SIGINT) + statistics.ready = true; } - - Strings tests_tags = options.count("tags") ? options["tags"].as() : Strings({}); - Strings skip_tags = options.count("skip-tags") ? options["skip-tags"].as() : Strings({}); - Strings tests_names = options.count("names") ? options["names"].as() : Strings({}); - Strings skip_names = options.count("skip-names") ? options["skip-names"].as() : Strings({}); - Strings tests_names_regexp = options.count("names-regexp") ? options["names-regexp"].as() : Strings({}); - Strings skip_names_regexp = options.count("skip-names-regexp") ? options["skip-names-regexp"].as() : Strings({}); - - auto timeouts = DB::ConnectionTimeouts::getTCPTimeoutsWithoutFailover(DB::Settings()); - - DB::UseSSL use_ssl; - - DB::PerformanceTest performance_test( - options["host"].as(), - options["port"].as(), - options.count("secure"), - options["database"].as(), - options["user"].as(), - options["password"].as(), - options.count("lite") > 0, - options["profiles-file"].as(), - std::move(input_files), - std::move(tests_tags), - std::move(skip_tags), - std::move(tests_names), - std::move(skip_names), - std::move(tests_names_regexp), - std::move(skip_names_regexp), - timeouts); - return performance_test.run(); } -catch (...) -{ - std::cout << DB::getCurrentExceptionMessage(/*with stacktrace = */ true) << std::endl; - int code = DB::getCurrentExceptionCode(); - return code ? code : 1; + + } diff --git a/dbms/programs/performance-test/PerformanceTest.h b/dbms/programs/performance-test/PerformanceTest.h new file mode 100644 index 00000000000..cebddacfc56 --- /dev/null +++ b/dbms/programs/performance-test/PerformanceTest.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include +#include "PerformanceTestInfo.h" + + +namespace DB +{ + +using XMLConfiguration = Poco::Util::XMLConfiguration; +using XMLConfigurationPtr = Poco::AutoPtr; +using QueriesWithIndexes = std::vector>; + + +class PerformanceTest +{ +public: + + PerformanceTest( + const XMLConfigurationPtr & config_, + Connection & connection_, + InterruptListener & interrupt_listener_, + const PerformanceTestInfo & test_info_); + + bool checkPreconditions() const; + std::vector execute(); + + const PerformanceTestInfo & getTestInfo() const + { + return test_info; + } + +private: + void runQueries( + const QueriesWithIndexes & queries_with_indexes, + std::vector & statistics_by_run); + + +private: + XMLConfigurationPtr config; + Connection & connection; + InterruptListener & interrupt_listener; + + PerformanceTestInfo test_info; + +}; +} diff --git a/dbms/programs/performance-test/PerformanceTestInfo.cpp b/dbms/programs/performance-test/PerformanceTestInfo.cpp new file mode 100644 index 00000000000..c7a45921eb2 --- /dev/null +++ b/dbms/programs/performance-test/PerformanceTestInfo.cpp @@ -0,0 +1,271 @@ +#include "PerformanceTestInfo.h" +#include +#include +#include +#include +#include +#include "applySubstitutions.h" + +namespace DB +{ +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +extern const int FILE_DOESNT_EXIST; +} + +namespace +{ + +void extractSettings( + const XMLConfigurationPtr & config, + const String & key, + const Strings & settings_list, + std::map & settings_to_apply) +{ + for (const String & setup : settings_list) + { + if (setup == "profile") + continue; + + String value = config->getString(key + "." + setup); + if (value.empty()) + value = "true"; + + settings_to_apply[setup] = value; + } +} + +void checkMetricsInput(const std::vector & metrics, ExecutionType exec_type) +{ + std::vector loop_metrics = { + "min_time", "quantiles", "total_time", + "queries_per_second", "rows_per_second", + "bytes_per_second"}; + + std::vector non_loop_metrics = { + "max_rows_per_second", "max_bytes_per_second", + "avg_rows_per_second", "avg_bytes_per_second"}; + + if (exec_type == ExecutionType::Loop) + { + for (const std::string & metric : metrics) + { + auto non_loop_pos = + std::find(non_loop_metrics.begin(), non_loop_metrics.end(), metric); + + if (non_loop_pos != non_loop_metrics.end()) + throw Exception("Wrong type of metric for loop execution type (" + metric + ")", + ErrorCodes::BAD_ARGUMENTS); + } + } + else + { + for (const std::string & metric : metrics) + { + auto loop_pos = std::find(loop_metrics.begin(), loop_metrics.end(), metric); + if (loop_pos != loop_metrics.end()) + throw Exception( + "Wrong type of metric for non-loop execution type (" + metric + ")", + ErrorCodes::BAD_ARGUMENTS); + } + } +} + +} + + +namespace fs = boost::filesystem; + +PerformanceTestInfo::PerformanceTestInfo( + XMLConfigurationPtr config, + const std::string & profiles_file_) + : profiles_file(profiles_file_) +{ + applySettings(config); + extractQueries(config); + processSubstitutions(config); + getExecutionType(config); + getStopConditions(config); + getMetrics(config); +} + +void PerformanceTestInfo::applySettings(XMLConfigurationPtr config) +{ + if (config->has("settings")) + { + std::map settings_to_apply; + std::vector config_settings; + config->keys("settings", config_settings); + + auto settings_contain = [&config_settings] (const std::string & setting) + { + auto position = std::find(config_settings.begin(), config_settings.end(), setting); + return position != config_settings.end(); + + }; + /// Preprocess configuration file + if (settings_contain("profile")) + { + if (!profiles_file.empty()) + { + String profile_name = config->getString("settings.profile"); + XMLConfigurationPtr profiles_config(new XMLConfiguration(profiles_file)); + + std::vector profile_settings; + profiles_config->keys("profiles." + profile_name, profile_settings); + + extractSettings(profiles_config, "profiles." + profile_name, profile_settings, settings_to_apply); + } + } + + extractSettings(config, "settings", config_settings, settings_to_apply); + + /// This macro goes through all settings in the Settings.h + /// and, if found any settings in test's xml configuration + /// with the same name, sets its value to settings + std::map::iterator it; +#define EXTRACT_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) \ + it = settings_to_apply.find(#NAME); \ + if (it != settings_to_apply.end()) \ + settings.set(#NAME, settings_to_apply[#NAME]); + + APPLY_FOR_SETTINGS(EXTRACT_SETTING) + +#undef EXTRACT_SETTING + + if (settings_contain("average_rows_speed_precision")) + TestStats::avg_rows_speed_precision = + config->getDouble("settings.average_rows_speed_precision"); + + if (settings_contain("average_bytes_speed_precision")) + TestStats::avg_bytes_speed_precision = + config->getDouble("settings.average_bytes_speed_precision"); + } +} + +void PerformanceTestInfo::extractQueries(XMLConfigurationPtr config) +{ + if (config->has("query")) + queries = getMultipleValuesFromConfig(*config, "", "query"); + + if (config->has("query_file")) + { + const String filename = config->getString("query_file"); + if (filename.empty()) + throw Exception("Empty file name", ErrorCodes::BAD_ARGUMENTS); + + bool tsv = fs::path(filename).extension().string() == ".tsv"; + + ReadBufferFromFile query_file(filename); + std::string query; + + if (tsv) + { + while (!query_file.eof()) + { + readEscapedString(query, query_file); + assertChar('\n', query_file); + queries.push_back(query); + } + } + else + { + readStringUntilEOF(query, query_file); + queries.push_back(query); + } + } + + if (queries.empty()) + throw Exception("Did not find any query to execute: " + test_name, + ErrorCodes::BAD_ARGUMENTS); +} + +void PerformanceTestInfo::processSubstitutions(XMLConfigurationPtr config) +{ + if (config->has("substitutions")) + { + /// Make "subconfig" of inner xml block + ConfigurationPtr substitutions_view(config->createView("substitutions")); + constructSubstitutions(substitutions_view, substitutions); + + auto queries_pre_format = queries; + queries.clear(); + for (const auto & query : queries_pre_format) + { + auto formatted = formatQueries(query, substitutions); + queries.insert(queries.end(), formatted.begin(), formatted.end()); + } + } +} + +void PerformanceTestInfo::getExecutionType(XMLConfigurationPtr config) +{ + if (!config->has("type")) + throw Exception("Missing type property in config: " + test_name, + ErrorCodes::BAD_ARGUMENTS); + + String config_exec_type = config->getString("type"); + if (config_exec_type == "loop") + exec_type = ExecutionType::Loop; + else if (config_exec_type == "once") + exec_type = ExecutionType::Once; + else + throw Exception("Unknown type " + config_exec_type + " in :" + test_name, + ErrorCodes::BAD_ARGUMENTS); +} + + +void PerformanceTestInfo::getStopConditions(XMLConfigurationPtr config) +{ + TestStopConditions stop_conditions_template; + if (config->has("stop_conditions")) + { + ConfigurationPtr stop_conditions_config(config->createView("stop_conditions")); + stop_conditions_template.loadFromConfig(stop_conditions_config); + } + + if (stop_conditions_template.empty()) + throw Exception("No termination conditions were found in config", + ErrorCodes::BAD_ARGUMENTS); + + for (size_t i = 0; i < times_to_run * queries.size(); ++i) + stop_conditions_by_run.push_back(stop_conditions_template); + + times_to_run = config->getUInt("times_to_run", 1); +} + + +void PerformanceTestInfo::getMetrics(XMLConfigurationPtr config) +{ + ConfigurationPtr metrics_view(config->createView("metrics")); + metrics_view->keys(metrics); + + if (config->has("main_metric")) + { + std::vector main_metrics; + config->keys("main_metric", main_metrics); + if (main_metrics.size()) + main_metric = main_metrics[0]; + } + + if (!main_metric.empty()) + { + if (std::find(metrics.begin(), metrics.end(), main_metric) == metrics.end()) + metrics.push_back(main_metric); + } + else + { + if (metrics.empty()) + throw Exception("You shoud specify at least one metric", + ErrorCodes::BAD_ARGUMENTS); + main_metric = metrics[0]; + } + + if (metrics.size() > 0) + checkMetricsInput(metrics, exec_type); +} + +} diff --git a/dbms/programs/performance-test/PerformanceTestInfo.h b/dbms/programs/performance-test/PerformanceTestInfo.h new file mode 100644 index 00000000000..c788a4f989a --- /dev/null +++ b/dbms/programs/performance-test/PerformanceTestInfo.h @@ -0,0 +1,52 @@ +#pragma once +#include +#include +#include +#include +#include +#include + +#include "StopConditionsSet.h" +#include "TestStopConditions.h" +#include "TestStats.h" + +namespace DB +{ +enum class ExecutionType +{ + Loop, + Once +}; + +using XMLConfiguration = Poco::Util::XMLConfiguration; +using XMLConfigurationPtr = Poco::AutoPtr; +using StringToVector = std::map>; + +class PerformanceTestInfo +{ +public: + PerformanceTestInfo(XMLConfigurationPtr config, const std::string & profiles_file_); + + std::string test_name; + std::string main_metric; + + std::vector queries; + std::vector metrics; + + Settings settings; + ExecutionType exec_type; + StringToVector substitutions; + size_t times_to_run; + std::string profiles_file; + std::vector stop_conditions_by_run; + +private: + void applySettings(XMLConfigurationPtr config); + void extractQueries(XMLConfigurationPtr config); + void processSubstitutions(XMLConfigurationPtr config); + void getExecutionType(XMLConfigurationPtr config); + void getStopConditions(XMLConfigurationPtr config); + void getMetrics(XMLConfigurationPtr config); +}; + +} diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp new file mode 100644 index 00000000000..29cb91afac5 --- /dev/null +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -0,0 +1,400 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "JSONString.h" +#include "StopConditionsSet.h" +#include "TestStopConditions.h" +#include "TestStats.h" +#include "ConfigPreprocessor.h" +#include "PerformanceTest.h" +#include "ReportBuilder.h" + +#ifndef __clang__ +#pragma GCC optimize("-fno-var-tracking-assignments") +#endif + + +/** Tests launcher for ClickHouse. + * The tool walks through given or default folder in order to find files with + * tests' descriptions and launches it. + */ +namespace fs = boost::filesystem; +using String = std::string; + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int FILE_DOESNT_EXIST; +} + + +using ConfigurationPtr = Poco::AutoPtr; + +class PerformanceTestSuite : public Poco::Util::Application +{ +public: + using Strings = std::vector; + + PerformanceTestSuite(const String & host_, + const UInt16 port_, + const bool secure_, + const String & default_database_, + const String & user_, + const String & password_, + const bool lite_output_, + const String & profiles_file_, + Strings && input_files_, + Strings && tests_tags_, + Strings && skip_tags_, + Strings && tests_names_, + Strings && skip_names_, + Strings && tests_names_regexp_, + Strings && skip_names_regexp_, + const ConnectionTimeouts & timeouts) + : connection(host_, port_, default_database_, user_, password_, timeouts, "performance-test", Protocol::Compression::Enable, secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable), + lite_output(lite_output_), + profiles_file(profiles_file_), + input_files(input_files_), + tests_tags(std::move(tests_tags_)), + skip_tags(std::move(skip_tags_)), + tests_names(std::move(tests_names_)), + skip_names(std::move(skip_names_)), + tests_names_regexp(std::move(tests_names_regexp_)), + skip_names_regexp(std::move(skip_names_regexp_)) + { + if (input_files.size() < 1) + { + throw DB::Exception("No tests were specified", DB::ErrorCodes::BAD_ARGUMENTS); + } + } + + void initialize(Poco::Util::Application & self [[maybe_unused]]) + { + std::string home_path; + const char * home_path_cstr = getenv("HOME"); + if (home_path_cstr) + home_path = home_path_cstr; + configReadClient(Poco::Util::Application::instance().config(), home_path); + } + + int main(const std::vector < std::string > & /* args */) + { + std::string name; + UInt64 version_major; + UInt64 version_minor; + UInt64 version_patch; + UInt64 version_revision; + connection.getServerVersion(name, version_major, version_minor, version_patch, version_revision); + + std::stringstream ss; + ss << version_major << "." << version_minor << "." << version_patch; + server_version = ss.str(); + + report_builder = std::make_shared(server_version); + + processTestsConfigurations(input_files); + + return 0; + } + +private: + std::string test_name; + + const Strings & tests_tags; + const Strings & tests_names; + const Strings & tests_names_regexp; + const Strings & skip_tags; + const Strings & skip_names; + const Strings & skip_names_regexp; + + std::shared_ptr report_builder; + using Query = String; + using Queries = std::vector; + using QueriesWithIndexes = std::vector>; + Queries queries; + + Connection connection; + std::string server_version; + + using Keys = std::vector; + + InterruptListener interrupt_listener; + + using XMLConfiguration = Poco::Util::XMLConfiguration; + using XMLConfigurationPtr = Poco::AutoPtr; + + using Paths = std::vector; + using StringToVector = std::map>; + using StringToMap = std::map; + StringToMap substitutions; + + + std::vector stop_conditions_by_run; + String main_metric; + bool lite_output; + String profiles_file; + + Strings input_files; + std::vector tests_configurations; + + + enum class ExecutionType + { + Loop, + Once + }; + ExecutionType exec_type; + + + size_t times_to_run = 1; + std::vector statistics_by_run; + + void processTestsConfigurations(const Paths & paths) + { + ConfigPreprocessor config_prep(paths); + tests_configurations = config_prep.processConfig( + tests_tags, + tests_names, + tests_names_regexp, + skip_tags, + skip_names, + skip_names_regexp); + + if (tests_configurations.size()) + { + Strings outputs; + + for (auto & test_config : tests_configurations) + { + String output = runTest(test_config); + if (lite_output) + std::cout << output; + else + outputs.push_back(output); + } + + if (!lite_output && outputs.size()) + { + std::cout << "[" << std::endl; + + for (size_t i = 0; i != outputs.size(); ++i) + { + std::cout << outputs[i]; + if (i != outputs.size() - 1) + std::cout << ","; + + std::cout << std::endl; + } + + std::cout << "]" << std::endl; + } + } + } + + String runTest(XMLConfigurationPtr & test_config) + { + //test_name = test_config->getString("name"); + //std::cerr << "Running: " << test_name << "\n"; + + PerformanceTestInfo info(test_config, profiles_file); + PerformanceTest current(test_config, connection, interrupt_listener, info); + current.checkPreconditions(); + + auto result = current.execute(); + + + if (lite_output) + return report_builder->buildCompactReport(info, result); + else + return report_builder->buildFullReport(info, result); + } + +}; +} + +static void getFilesFromDir(const fs::path & dir, std::vector & input_files, const bool recursive = false) +{ + if (dir.extension().string() == ".xml") + std::cerr << "Warning: '" + dir.string() + "' is a directory, but has .xml extension" << std::endl; + + fs::directory_iterator end; + for (fs::directory_iterator it(dir); it != end; ++it) + { + const fs::path file = (*it); + if (recursive && fs::is_directory(file)) + getFilesFromDir(file, input_files, recursive); + else if (!fs::is_directory(file) && file.extension().string() == ".xml") + input_files.push_back(file.string()); + } +} + + +int mainEntryClickHousePerformanceTest(int argc, char ** argv) +try +{ + using boost::program_options::value; + using Strings = std::vector; + + boost::program_options::options_description desc("Allowed options"); + desc.add_options() + ("help", "produce help message") + ("lite", "use lite version of output") + ("profiles-file", value()->default_value(""), "Specify a file with global profiles") + ("host,h", value()->default_value("localhost"), "") + ("port", value()->default_value(9000), "") + ("secure,s", "Use TLS connection") + ("database", value()->default_value("default"), "") + ("user", value()->default_value("default"), "") + ("password", value()->default_value(""), "") + ("tags", value()->multitoken(), "Run only tests with tag") + ("skip-tags", value()->multitoken(), "Do not run tests with tag") + ("names", value()->multitoken(), "Run tests with specific name") + ("skip-names", value()->multitoken(), "Do not run tests with name") + ("names-regexp", value()->multitoken(), "Run tests with names matching regexp") + ("skip-names-regexp", value()->multitoken(), "Do not run tests with names matching regexp") + ("recursive,r", "Recurse in directories to find all xml's"); + + /// These options will not be displayed in --help + boost::program_options::options_description hidden("Hidden options"); + hidden.add_options() + ("input-files", value>(), ""); + + /// But they will be legit, though. And they must be given without name + boost::program_options::positional_options_description positional; + positional.add("input-files", -1); + + boost::program_options::options_description cmdline_options; + cmdline_options.add(desc).add(hidden); + + boost::program_options::variables_map options; + boost::program_options::store( + boost::program_options::command_line_parser(argc, argv).options(cmdline_options).positional(positional).run(), options); + boost::program_options::notify(options); + + if (options.count("help")) + { + std::cout << "Usage: " << argv[0] << " [options] [test_file ...] [tests_folder]\n"; + std::cout << desc << "\n"; + return 0; + } + + Strings input_files; + bool recursive = options.count("recursive"); + + if (!options.count("input-files")) + { + std::cerr << "Trying to find test scenario files in the current folder..."; + fs::path curr_dir("."); + + getFilesFromDir(curr_dir, input_files, recursive); + + if (input_files.empty()) + { + std::cerr << std::endl; + throw DB::Exception("Did not find any xml files", DB::ErrorCodes::BAD_ARGUMENTS); + } + else + std::cerr << " found " << input_files.size() << " files." << std::endl; + } + else + { + input_files = options["input-files"].as(); + Strings collected_files; + + for (const String & filename : input_files) + { + fs::path file(filename); + + if (!fs::exists(file)) + throw DB::Exception("File '" + filename + "' does not exist", DB::ErrorCodes::FILE_DOESNT_EXIST); + + if (fs::is_directory(file)) + { + getFilesFromDir(file, collected_files, recursive); + } + else + { + if (file.extension().string() != ".xml") + throw DB::Exception("File '" + filename + "' does not have .xml extension", DB::ErrorCodes::BAD_ARGUMENTS); + collected_files.push_back(filename); + } + } + + input_files = std::move(collected_files); + } + + Strings tests_tags = options.count("tags") ? options["tags"].as() : Strings({}); + Strings skip_tags = options.count("skip-tags") ? options["skip-tags"].as() : Strings({}); + Strings tests_names = options.count("names") ? options["names"].as() : Strings({}); + Strings skip_names = options.count("skip-names") ? options["skip-names"].as() : Strings({}); + Strings tests_names_regexp = options.count("names-regexp") ? options["names-regexp"].as() : Strings({}); + Strings skip_names_regexp = options.count("skip-names-regexp") ? options["skip-names-regexp"].as() : Strings({}); + + auto timeouts = DB::ConnectionTimeouts::getTCPTimeoutsWithoutFailover(DB::Settings()); + + DB::UseSSL use_ssl; + + DB::PerformanceTestSuite performance_test( + options["host"].as(), + options["port"].as(), + options.count("secure"), + options["database"].as(), + options["user"].as(), + options["password"].as(), + options.count("lite") > 0, + options["profiles-file"].as(), + std::move(input_files), + std::move(tests_tags), + std::move(skip_tags), + std::move(tests_names), + std::move(skip_names), + std::move(tests_names_regexp), + std::move(skip_names_regexp), + timeouts); + return performance_test.run(); +} +catch (...) +{ + std::cout << DB::getCurrentExceptionMessage(/*with stacktrace = */ true) << std::endl; + int code = DB::getCurrentExceptionCode(); + return code ? code : 1; +} diff --git a/dbms/programs/performance-test/ReportBuilder.cpp b/dbms/programs/performance-test/ReportBuilder.cpp new file mode 100644 index 00000000000..cd381aefa5e --- /dev/null +++ b/dbms/programs/performance-test/ReportBuilder.cpp @@ -0,0 +1,190 @@ +#include "ReportBuilder.h" +#include "JSONString.h" +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ +const std::regex QUOTE_REGEX{"\""}; +} + +ReportBuilder::ReportBuilder(const std::string & server_version_) + : server_version(server_version_) + , hostname(getFQDNOrHostName()) + , num_cores(getNumberOfPhysicalCPUCores()) + , num_threads(std::thread::hardware_concurrency()) + , ram(getMemoryAmount()) +{ +} + +std::string ReportBuilder::getCurrentTime() const +{ + return DateLUT::instance().timeToString(time(nullptr)); +} + +std::string ReportBuilder::buildFullReport( + const PerformanceTestInfo & test_info, + std::vector & stats) const +{ + JSONString json_output; + + json_output.set("hostname", hostname); + json_output.set("num_cores", num_cores); + json_output.set("num_threads", num_threads); + json_output.set("ram", ram); + json_output.set("server_version", server_version); + json_output.set("time", getCurrentTime()); + json_output.set("test_name", test_info.test_name); + json_output.set("main_metric", test_info.main_metric); + + auto has_metric = [&test_info] (const std::string & metric_name) + { + return std::find(test_info.metrics.begin(), + test_info.metrics.end(), metric_name) != test_info.metrics.end(); + }; + + if (test_info.substitutions.size()) + { + JSONString json_parameters(2); /// here, 2 is the size of \t padding + + for (auto it = test_info.substitutions.begin(); it != test_info.substitutions.end(); ++it) + { + String parameter = it->first; + std::vector values = it->second; + + String array_string = "["; + for (size_t i = 0; i != values.size(); ++i) + { + array_string += '"' + std::regex_replace(values[i], QUOTE_REGEX, "\\\"") + '"'; + if (i != values.size() - 1) + { + array_string += ", "; + } + } + array_string += ']'; + + json_parameters.set(parameter, array_string); + } + + json_output.set("parameters", json_parameters.asString()); + } + + std::vector run_infos; + for (size_t query_index = 0; query_index < test_info.queries.size(); ++query_index) + { + for (size_t number_of_launch = 0; number_of_launch < test_info.times_to_run; ++number_of_launch) + { + size_t stat_index = number_of_launch * test_info.queries.size() + query_index; + TestStats & statistics = stats[stat_index]; + + if (!statistics.ready) + continue; + + JSONString runJSON; + + auto query = std::regex_replace(test_info.queries[query_index], QUOTE_REGEX, "\\\""); + runJSON.set("query", query); + if (!statistics.exception.empty()) + runJSON.set("exception", statistics.exception); + + if (test_info.exec_type == ExecutionType::Loop) + { + /// in seconds + if (has_metric("min_time")) + runJSON.set("min_time", statistics.min_time / double(1000)); + + if (has_metric("quantiles")) + { + JSONString quantiles(4); /// here, 4 is the size of \t padding + for (double percent = 10; percent <= 90; percent += 10) + { + String quantile_key = std::to_string(percent / 100.0); + while (quantile_key.back() == '0') + quantile_key.pop_back(); + + quantiles.set(quantile_key, + statistics.sampler.quantileInterpolated(percent / 100.0)); + } + quantiles.set("0.95", + statistics.sampler.quantileInterpolated(95 / 100.0)); + quantiles.set("0.99", + statistics.sampler.quantileInterpolated(99 / 100.0)); + quantiles.set("0.999", + statistics.sampler.quantileInterpolated(99.9 / 100.0)); + quantiles.set("0.9999", + statistics.sampler.quantileInterpolated(99.99 / 100.0)); + + runJSON.set("quantiles", quantiles.asString()); + } + + if (has_metric("total_time")) + runJSON.set("total_time", statistics.total_time); + + if (has_metric("queries_per_second")) + runJSON.set("queries_per_second", + double(statistics.queries) / statistics.total_time); + + if (has_metric("rows_per_second")) + runJSON.set("rows_per_second", + double(statistics.total_rows_read) / statistics.total_time); + + if (has_metric("bytes_per_second")) + runJSON.set("bytes_per_second", + double(statistics.total_bytes_read) / statistics.total_time); + } + else + { + if (has_metric("max_rows_per_second")) + runJSON.set("max_rows_per_second", statistics.max_rows_speed); + + if (has_metric("max_bytes_per_second")) + runJSON.set("max_bytes_per_second", statistics.max_bytes_speed); + + if (has_metric("avg_rows_per_second")) + runJSON.set("avg_rows_per_second", statistics.avg_rows_speed_value); + + if (has_metric("avg_bytes_per_second")) + runJSON.set("avg_bytes_per_second", statistics.avg_bytes_speed_value); + } + + run_infos.push_back(runJSON); + } + } + + json_output.set("runs", run_infos); + + return json_output.asString(); +} + +std::string ReportBuilder::buildCompactReport( + const PerformanceTestInfo & test_info, + std::vector & stats) const +{ + + String output; + + for (size_t query_index = 0; query_index < test_info.queries.size(); ++query_index) + { + for (size_t number_of_launch = 0; number_of_launch < test_info.times_to_run; ++number_of_launch) + { + if (test_info.queries.size() > 1) + output += "query \"" + test_info.queries[query_index] + "\", "; + + output += "run " + std::to_string(number_of_launch + 1) + ": "; + output += test_info.main_metric + " = "; + size_t index = number_of_launch * test_info.queries.size() + query_index; + output += stats[index].getStatisticByName(test_info.main_metric); + output += "\n"; + } + } + return output; +} + + +} diff --git a/dbms/programs/performance-test/ReportBuilder.h b/dbms/programs/performance-test/ReportBuilder.h new file mode 100644 index 00000000000..0972061e27a --- /dev/null +++ b/dbms/programs/performance-test/ReportBuilder.h @@ -0,0 +1,30 @@ +#pragma once +#include "PerformanceTestInfo.h" + +namespace DB +{ + +class ReportBuilder +{ +public: + explicit ReportBuilder(const std::string & server_version_); + std::string buildFullReport( + const PerformanceTestInfo & test_info, + std::vector & stats) const; + + std::string buildCompactReport( + const PerformanceTestInfo & test_info, + std::vector & stats) const; +private: + std::string server_version; + std::string hostname; + size_t num_cores; + size_t num_threads; + size_t ram; + +private: + std::string getCurrentTime() const; + +}; + +} diff --git a/dbms/programs/performance-test/TestStats.cpp b/dbms/programs/performance-test/TestStats.cpp index 163aefdc98d..bc23ef17472 100644 --- a/dbms/programs/performance-test/TestStats.cpp +++ b/dbms/programs/performance-test/TestStats.cpp @@ -157,6 +157,7 @@ void TestStats::clear() total_bytes_read = 0; last_query_rows_read = 0; last_query_bytes_read = 0; + got_SIGINT = false; min_time = std::numeric_limits::max(); total_time = 0; diff --git a/dbms/programs/performance-test/TestStats.h b/dbms/programs/performance-test/TestStats.h index 41a8efc3beb..5b8dd773566 100644 --- a/dbms/programs/performance-test/TestStats.h +++ b/dbms/programs/performance-test/TestStats.h @@ -51,6 +51,8 @@ struct TestStats bool ready = false; // check if a query wasn't interrupted by SIGINT String exception; + bool got_SIGINT = false; + String getStatisticByName(const String & statistic_name); void update_min_time(UInt64 min_time_candidate); diff --git a/dbms/programs/performance-test/applySubstitutions.cpp b/dbms/programs/performance-test/applySubstitutions.cpp new file mode 100644 index 00000000000..915d9ba7230 --- /dev/null +++ b/dbms/programs/performance-test/applySubstitutions.cpp @@ -0,0 +1,82 @@ +#include "applySubstitutions.h" +#include +#include + +namespace DB +{ + +void constructSubstitutions(ConfigurationPtr & substitutions_view, StringToVector & out_substitutions) +{ + std::vector xml_substitutions; + substitutions_view->keys(xml_substitutions); + + for (size_t i = 0; i != xml_substitutions.size(); ++i) + { + const ConfigurationPtr xml_substitution(substitutions_view->createView("substitution[" + std::to_string(i) + "]")); + + /// Property values for substitution will be stored in a vector + /// accessible by property name + std::vector xml_values; + xml_substitution->keys("values", xml_values); + + String name = xml_substitution->getString("name"); + + for (size_t j = 0; j != xml_values.size(); ++j) + { + out_substitutions[name].push_back(xml_substitution->getString("values.value[" + std::to_string(j) + "]")); + } + } +} + +/// Recursive method which goes through all substitution blocks in xml +/// and replaces property {names} by their values +void runThroughAllOptionsAndPush(StringToVector::iterator substitutions_left, + StringToVector::iterator substitutions_right, + const String & template_query, + std::vector & out_queries) +{ + if (substitutions_left == substitutions_right) + { + out_queries.push_back(template_query); /// completely substituted query + return; + } + + String substitution_mask = "{" + substitutions_left->first + "}"; + + if (template_query.find(substitution_mask) == String::npos) /// nothing to substitute here + { + runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, template_query, out_queries); + return; + } + + for (const String & value : substitutions_left->second) + { + /// Copy query string for each unique permutation + std::string query = template_query; + size_t substr_pos = 0; + + while (substr_pos != String::npos) + { + substr_pos = query.find(substitution_mask); + + if (substr_pos != String::npos) + query.replace(substr_pos, substitution_mask.length(), value); + } + + runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, query, out_queries); + } +} + +std::vector formatQueries(const String & query, StringToVector substitutions_to_generate) +{ + std::vector queries_res; + runThroughAllOptionsAndPush( + substitutions_to_generate.begin(), + substitutions_to_generate.end(), + query, + queries_res); + return queries_res; +} + + +} diff --git a/dbms/programs/performance-test/applySubstitutions.h b/dbms/programs/performance-test/applySubstitutions.h new file mode 100644 index 00000000000..7d50e4bb09a --- /dev/null +++ b/dbms/programs/performance-test/applySubstitutions.h @@ -0,0 +1,18 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +using StringToVector = std::map>; +using ConfigurationPtr = Poco::AutoPtr; + +void constructSubstitutions(ConfigurationPtr & substitutions_view, StringToVector & out_substitutions); + +std::vector formatQueries(const String & query, StringToVector substitutions_to_generate); + +} diff --git a/dbms/programs/performance-test/executeQuery.cpp b/dbms/programs/performance-test/executeQuery.cpp new file mode 100644 index 00000000000..45487acf3b9 --- /dev/null +++ b/dbms/programs/performance-test/executeQuery.cpp @@ -0,0 +1,72 @@ +#include "executeQuery.h" +#include +#include +#include +namespace DB +{ +namespace +{ + +void checkFulfilledConditionsAndUpdate( + const Progress & progress, RemoteBlockInputStream & stream, + TestStats & statistics, TestStopConditions & stop_conditions, + InterruptListener & interrupt_listener) +{ + statistics.add(progress.rows, progress.bytes); + + stop_conditions.reportRowsRead(statistics.total_rows_read); + stop_conditions.reportBytesReadUncompressed(statistics.total_bytes_read); + stop_conditions.reportTotalTime(statistics.watch.elapsed() / (1000 * 1000)); + stop_conditions.reportMinTimeNotChangingFor(statistics.min_time_watch.elapsed() / (1000 * 1000)); + stop_conditions.reportMaxSpeedNotChangingFor(statistics.max_rows_speed_watch.elapsed() / (1000 * 1000)); + stop_conditions.reportAverageSpeedNotChangingFor(statistics.avg_rows_speed_watch.elapsed() / (1000 * 1000)); + + if (stop_conditions.areFulfilled()) + { + statistics.last_query_was_cancelled = true; + stream.cancel(false); + } + + if (interrupt_listener.check()) + { + statistics.got_SIGINT = true; + statistics.last_query_was_cancelled = true; + stream.cancel(false); + } +} + +} + +void executeQuery( + Connection & connection, + const std::string & query, + TestStats & statistics, + TestStopConditions & stop_conditions, + InterruptListener & interrupt_listener) +{ + statistics.watch_per_query.restart(); + statistics.last_query_was_cancelled = false; + statistics.last_query_rows_read = 0; + statistics.last_query_bytes_read = 0; + + Settings settings; + Context global_context = Context::createGlobal(); + RemoteBlockInputStream stream(connection, query, {}, global_context, &settings); + + stream.setProgressCallback( + [&](const Progress & value) + { + checkFulfilledConditionsAndUpdate( + value, stream, statistics, + stop_conditions, interrupt_listener); + }); + stream.readPrefix(); + while (Block block = stream.read()); + stream.readSuffix(); + + if (!statistics.last_query_was_cancelled) + statistics.updateQueryInfo(); + + statistics.setTotalTime(); +} +} diff --git a/dbms/programs/performance-test/executeQuery.h b/dbms/programs/performance-test/executeQuery.h new file mode 100644 index 00000000000..27272842f02 --- /dev/null +++ b/dbms/programs/performance-test/executeQuery.h @@ -0,0 +1,16 @@ +#pragma once +#include +#include "TestStats.h" +#include "TestStopConditions.h" +#include +#include + +namespace DB +{ +void executeQuery( + Connection & connection, + const std::string & query, + TestStats & statistics, + TestStopConditions & stop_conditions, + InterruptListener & interrupt_listener); +} From 253ac93459f80610a864b033893d36ab5f7c7380 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Jan 2019 21:44:30 +0300 Subject: [PATCH 072/324] Addition to prev. revision #4150 --- dbms/src/Common/CurrentThread.cpp | 3 +++ dbms/src/DataStreams/AsynchronousBlockInputStream.cpp | 2 +- .../MergingAggregatedMemoryEfficientBlockInputStream.cpp | 2 +- dbms/src/Interpreters/ThreadStatusExt.cpp | 1 - 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp index c3e0cae9571..b6d161af67e 100644 --- a/dbms/src/Common/CurrentThread.cpp +++ b/dbms/src/Common/CurrentThread.cpp @@ -71,6 +71,9 @@ std::shared_ptr CurrentThread::getInternalTextLogsQueue() ThreadGroupStatusPtr CurrentThread::getGroup() { + if (!current_thread) + return nullptr; + return get().getThreadGroup(); } diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp index ba31b45bfd2..558d15be456 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, thread_group=CurrentThread::getGroup()] () + pool.schedule([this, thread_group = CurrentThread::getGroup()] () { CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index f226da9e442..bde030d8afa 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -195,7 +195,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start() */ for (size_t i = 0; i < merging_threads; ++i) - pool.schedule([this, thread_group=CurrentThread::getGroup()] () { mergeThread(thread_group); }); + pool.schedule([this, thread_group = CurrentThread::getGroup()] () { mergeThread(thread_group); }); } } diff --git a/dbms/src/Interpreters/ThreadStatusExt.cpp b/dbms/src/Interpreters/ThreadStatusExt.cpp index 987365cb3c4..669322a2509 100644 --- a/dbms/src/Interpreters/ThreadStatusExt.cpp +++ b/dbms/src/Interpreters/ThreadStatusExt.cpp @@ -37,7 +37,6 @@ String ThreadStatus::getQueryID() void CurrentThread::defaultThreadDeleter() { ThreadStatus & thread = CurrentThread::get(); - LOG_TRACE(thread.log, "Thread " << thread.thread_number << " exited"); thread.detachQuery(true, true); } From e1f818a1b6cd6f7c3f69d693307e5349f377a583 Mon Sep 17 00:00:00 2001 From: Alexander GQ Gerasiov Date: Wed, 16 Jan 2019 01:39:01 +0300 Subject: [PATCH 073/324] Enable link time optimization (-flto). This reduce target's size by 30%. Signed-off-by: Alexander GQ Gerasiov --- CMakeLists.txt | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 6bd0aae5fd1..8b1903a43d3 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -1,8 +1,23 @@ project (ClickHouse) -cmake_minimum_required (VERSION 3.3) +cmake_minimum_required (VERSION 3.9) set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake/Modules/") +set(ENABLE_IPO OFF CACHE STRING "Enable inter-procedural optimization (aka LTO)") + +if (ENABLE_IPO) +include(CheckIPOSupported) +check_ipo_supported(RESULT IPO_SUPPORTED OUTPUT IPO_NOT_SUPPORTED) +if(IPO_SUPPORTED) + message(STATUS "IPO/LTO is supported, enabling") + set(CMAKE_INTERPROCEDURAL_OPTIMIZATION TRUE) +else() + message(STATUS "IPO/LTO is not supported: <${IPO_NOT_SUPPORTED}>") +endif() +else() + message(STATUS "IPO/LTO not enabled.") +endif() + if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") # Require at least gcc 7 if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS 7 AND NOT CMAKE_VERSION VERSION_LESS 2.8.9) From dd16a012a4f1423eafb8be05b3302960a36b7cab Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 25 Jan 2019 22:18:05 +0300 Subject: [PATCH 074/324] IPO fixes --- CMakeLists.txt | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 8b1903a43d3..f7297f31ed8 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -1,19 +1,19 @@ project (ClickHouse) -cmake_minimum_required (VERSION 3.9) +cmake_minimum_required (VERSION 3.3) set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake/Modules/") -set(ENABLE_IPO OFF CACHE STRING "Enable inter-procedural optimization (aka LTO)") - -if (ENABLE_IPO) -include(CheckIPOSupported) -check_ipo_supported(RESULT IPO_SUPPORTED OUTPUT IPO_NOT_SUPPORTED) -if(IPO_SUPPORTED) - message(STATUS "IPO/LTO is supported, enabling") - set(CMAKE_INTERPROCEDURAL_OPTIMIZATION TRUE) -else() - message(STATUS "IPO/LTO is not supported: <${IPO_NOT_SUPPORTED}>") -endif() +option(ENABLE_IPO "Enable inter-procedural optimization (aka LTO)" OFF) # need cmake 3.9+ +if(ENABLE_IPO) + cmake_policy(SET CMP0069 NEW) + include(CheckIPOSupported) + check_ipo_supported(RESULT IPO_SUPPORTED OUTPUT IPO_NOT_SUPPORTED) + if(IPO_SUPPORTED) + message(STATUS "IPO/LTO is supported, enabling") + set(CMAKE_INTERPROCEDURAL_OPTIMIZATION TRUE) + else() + message(STATUS "IPO/LTO is not supported: <${IPO_NOT_SUPPORTED}>") + endif() else() message(STATUS "IPO/LTO not enabled.") endif() From 6964fb47ff5074d746a1b68b1ebc32fe558fe2a8 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 25 Jan 2019 22:29:01 +0300 Subject: [PATCH 075/324] BUILD_DETERMINISTIC --- .../StorageSystemBuildOptions.generated.cpp.in | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 5c72545ab44..1cb420ce7b6 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -1,15 +1,23 @@ // .cpp autogenerated by cmake +#cmakedefine01 BUILD_DETERMINISTIC + const char * auto_config_build[] { "VERSION_FULL", "@VERSION_FULL@", "VERSION_DESCRIBE", "@VERSION_DESCRIBE@", + "VERSION_INTEGER", "@VERSION_INTEGER@", + +#if BUILD_DETERMINISTIC + "SYSTEM", "@CMAKE_SYSTEM_NAME@", +#else "VERSION_GITHASH", "@VERSION_GITHASH@", "VERSION_REVISION", "@VERSION_REVISION@", - "VERSION_INTEGER", "@VERSION_INTEGER@", "BUILD_DATE", "@BUILD_DATE@", - "BUILD_TYPE", "@CMAKE_BUILD_TYPE@", "SYSTEM", "@CMAKE_SYSTEM@", +#endif + + "BUILD_TYPE", "@CMAKE_BUILD_TYPE@", "SYSTEM_PROCESSOR", "@CMAKE_SYSTEM_PROCESSOR@", "LIBRARY_ARCHITECTURE", "@CMAKE_LIBRARY_ARCHITECTURE@", "CMAKE_VERSION", "@CMAKE_VERSION@", From 038a48bb3833b2241c7afe9cab6d5b8bdc48c132 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 20:35:53 +0100 Subject: [PATCH 076/324] sumMap: implement sumMapWithOverflow --- .../AggregateFunctionSumMap.cpp | 47 ++++++++++++++++--- .../AggregateFunctionSumMap.h | 35 +++++++------- 2 files changed, 58 insertions(+), 24 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 02303b953d9..75cd62c00f1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -12,6 +12,37 @@ namespace DB namespace { +struct WithOverflowPolicy +{ + /// Overflow, meaning that the returned type is the same as the input type. + static DataTypePtr promoteType(const DataTypePtr & data_type) { return data_type; } +}; + +struct WithoutOverflowPolicy +{ + /// No overflow, meaning we promote the types if necessary. + static DataTypePtr promoteType(const DataTypePtr & data_type) + { + if (!data_type->canBePromoted()) + throw new Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return data_type->promoteNumericType(); + } +}; + +template +using SumMapWithOverflow = AggregateFunctionSumMap; + +template +using SumMapWithoutOverflow = AggregateFunctionSumMap; + +template +using SumMapFilteredWithOverflow = AggregateFunctionSumMapFiltered; + +template +using SumMapFilteredWithoutOverflow = AggregateFunctionSumMapFiltered; + using SumMapArgs = std::pair; SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) @@ -42,21 +73,23 @@ SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) return {std::move(keys_type), std::move(values_types)}; } +template