diff --git a/CMakeLists.txt b/CMakeLists.txt index ab13edb8940..e9a433d80b4 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -106,6 +106,10 @@ endif () if (COMPILER_CLANG) # clang: warning: argument unused during compilation: '-specs=/usr/share/dpkg/no-pie-compile.specs' [-Wunused-command-line-argument] set (COMMON_WARNING_FLAGS "${COMMON_WARNING_FLAGS} -Wno-unused-command-line-argument") + # generate ranges for fast "addr2line" search + if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") + set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges") + endif () endif () option (ENABLE_TESTS "Enables tests" ON) @@ -324,7 +328,7 @@ if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_L set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic ${LIBCXX_LIBS} ${EXCEPTION_HANDLING_LIBRARY} ${BUILTINS_LIB_PATH} -Wl,-Bdynamic") else () - set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic -lstdc++ ${EXCEPTION_HANDLING_LIBRARY} ${COVERAGE_OPTION} ${BUILTINS_LIB_PATH} -Wl,-Bdynamic") + set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic -lstdc++ -lstdc++fs ${EXCEPTION_HANDLING_LIBRARY} ${COVERAGE_OPTION} ${BUILTINS_LIB_PATH} -Wl,-Bdynamic") endif () # Linking with GLIBC prevents portability of binaries to older systems. diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 8292d16a999..19ca188438e 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -159,6 +159,15 @@ if (OS_FREEBSD) target_compile_definitions (clickhouse_common_io PUBLIC CLOCK_MONOTONIC_COARSE=CLOCK_MONOTONIC_FAST) endif () +if (USE_UNWIND) + target_compile_definitions (clickhouse_common_io PRIVATE USE_UNWIND=1) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${UNWIND_INCLUDE_DIR}) + + if (NOT USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING) + target_link_libraries (clickhouse_common_io PRIVATE ${UNWIND_LIBRARY}) + endif () +endif () + add_subdirectory(src/Common/ZooKeeper) add_subdirectory(src/Common/Config) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 108b2aeee20..563fff6a0bc 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -626,161 +626,164 @@ int Server::main(const std::vector & /*args*/) for (const auto & listen_host : listen_hosts) { - /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. - uint16_t listen_port = 0; - try + auto create_server = [&](const char * port_name, auto && func) { - /// HTTP - if (config().has("http_port")) - { - Poco::Net::ServerSocket socket; - listen_port = config().getInt("http_port"); - auto address = socket_bind_listen(socket, listen_host, listen_port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - servers.emplace_back(std::make_unique( - new HTTPHandlerFactory(*this, "HTTPHandler-factory"), - server_pool, - socket, - http_params)); + /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. + if (!config().has(port_name)) + return; - LOG_INFO(log, "Listening http://" + address.toString()); + auto port = config().getInt(port_name); + try + { + func(port); } - - /// HTTPS - if (config().has("https_port")) + catch (const Poco::Exception &) { -#if USE_POCO_NETSSL - Poco::Net::SecureServerSocket socket; - listen_port = config().getInt("https_port"); - auto address = socket_bind_listen(socket, listen_host, listen_port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - servers.emplace_back(std::make_unique( - new HTTPHandlerFactory(*this, "HTTPSHandler-factory"), - server_pool, - socket, - http_params)); + std::string message = "Listen [" + listen_host + "]:" + std::to_string(port) + " failed: " + getCurrentExceptionMessage(false); - LOG_INFO(log, "Listening https://" + address.toString()); + if (listen_try) + { + LOG_ERROR(log, message + << ". If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " + "specify not disabled IPv4 or IPv6 address to listen in element of configuration " + "file. Example for disabled IPv6: 0.0.0.0 ." + " Example for disabled IPv4: ::"); + } + else + { + throw Exception{message, ErrorCodes::NETWORK_ERROR}; + } + } + }; + + /// HTTP + create_server("http_port", [&](UInt16 port) + { + Poco::Net::ServerSocket socket; + auto address = socket_bind_listen(socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + servers.emplace_back(std::make_unique( + new HTTPHandlerFactory(*this, "HTTPHandler-factory"), + server_pool, + socket, + http_params)); + + LOG_INFO(log, "Listening http://" + address.toString()); + }); + + /// HTTPS + create_server("https_port", [&](UInt16 port) + { +#if USE_POCO_NETSSL + Poco::Net::SecureServerSocket socket; + auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + servers.emplace_back(std::make_unique( + new HTTPHandlerFactory(*this, "HTTPSHandler-factory"), + server_pool, + socket, + http_params)); + + LOG_INFO(log, "Listening https://" + address.toString()); #else - throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.", + throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + }); + + /// TCP + create_server("tcp_port", [&](UInt16 port) + { + Poco::Net::ServerSocket socket; + auto address = socket_bind_listen(socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + servers.emplace_back(std::make_unique( + new TCPHandlerFactory(*this), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + + LOG_INFO(log, "Listening for connections with native protocol (tcp): " + address.toString()); + }); + + /// TCP with SSL + create_server("tcp_port_secure", [&](UInt16 port) + { +#if USE_POCO_NETSSL + Poco::Net::SecureServerSocket socket; + auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + servers.emplace_back(std::make_unique( + new TCPHandlerFactory(*this, /* secure= */ true), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): " + address.toString()); +#else + throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + }); + + /// Interserver IO HTTP + create_server("interserver_http_port", [&](UInt16 port) + { + Poco::Net::ServerSocket socket; + auto address = socket_bind_listen(socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + servers.emplace_back(std::make_unique( + new InterserverIOHTTPHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), + server_pool, + socket, + http_params)); + + LOG_INFO(log, "Listening for replica communication (interserver) http://" + address.toString()); + }); + + create_server("interserver_https_port", [&](UInt16 port) + { +#if USE_POCO_NETSSL + Poco::Net::SecureServerSocket socket; + auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + servers.emplace_back(std::make_unique( + new InterserverIOHTTPHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), + server_pool, + socket, + http_params)); + + LOG_INFO(log, "Listening for secure replica communication (interserver) https://" + address.toString()); +#else + throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif - } + }); - /// TCP - if (config().has("tcp_port")) - { - Poco::Net::ServerSocket socket; - listen_port = config().getInt("tcp_port"); - auto address = socket_bind_listen(socket, listen_host, listen_port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - servers.emplace_back(std::make_unique( - new TCPHandlerFactory(*this), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - - LOG_INFO(log, "Listening for connections with native protocol (tcp): " + address.toString()); - } - - /// TCP with SSL - if (config().has("tcp_port_secure")) - { + create_server("mysql_port", [&](UInt16 port) + { #if USE_POCO_NETSSL - Poco::Net::SecureServerSocket socket; - listen_port = config().getInt("tcp_port_secure"); - auto address = socket_bind_listen(socket, listen_host, listen_port, /* secure = */ true); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - servers.emplace_back(std::make_unique( - new TCPHandlerFactory(*this, /* secure= */ true), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): " + address.toString()); + Poco::Net::ServerSocket socket; + auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + servers.emplace_back(std::make_unique( + new MySQLHandlerFactory(*this), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + + LOG_INFO(log, "Listening for MySQL compatibility protocol: " + address.toString()); #else - throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", + throw Exception{"SSL support for MySQL protocol is disabled because Poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif - } - - /// At least one of TCP and HTTP servers must be created. - if (servers.empty()) - throw Exception("No 'tcp_port' and 'http_port' is specified in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); - - /// Interserver IO HTTP - if (config().has("interserver_http_port")) - { - Poco::Net::ServerSocket socket; - listen_port = config().getInt("interserver_http_port"); - auto address = socket_bind_listen(socket, listen_host, listen_port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - servers.emplace_back(std::make_unique( - new InterserverIOHTTPHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), - server_pool, - socket, - http_params)); - - LOG_INFO(log, "Listening for replica communication (interserver) http://" + address.toString()); - } - - if (config().has("interserver_https_port")) - { -#if USE_POCO_NETSSL - Poco::Net::SecureServerSocket socket; - listen_port = config().getInt("interserver_https_port"); - auto address = socket_bind_listen(socket, listen_host, listen_port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - servers.emplace_back(std::make_unique( - new InterserverIOHTTPHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), - server_pool, - socket, - http_params)); - - LOG_INFO(log, "Listening for secure replica communication (interserver) https://" + address.toString()); -#else - throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", - ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - } - - if (config().has("mysql_port")) - { -#if USE_POCO_NETSSL - Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, config().getInt("mysql_port"), /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - servers.emplace_back(std::make_unique( - new MySQLHandlerFactory(*this), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - - LOG_INFO(log, "Listening for MySQL compatibility protocol: " + address.toString()); -#else - throw Exception{"SSL support for MySQL protocol is disabled because Poco library was built without NetSSL support.", - ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - } - } - catch (const Poco::Exception & e) - { - std::string message = "Listen [" + listen_host + "]:" + std::to_string(listen_port) + " failed: " + std::to_string(e.code()) + ": " + e.what() + ": " + e.message(); - if (listen_try) - LOG_ERROR(log, message - << " If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " - "specify not disabled IPv4 or IPv6 address to listen in element of configuration " - "file. Example for disabled IPv6: 0.0.0.0 ." - " Example for disabled IPv4: ::"); - else - throw Exception{message, ErrorCodes::NETWORK_ERROR}; - } + }); } if (servers.empty()) @@ -818,10 +821,13 @@ int Server::main(const std::vector & /*args*/) current_connections += server->currentConnections(); } - LOG_DEBUG(log, + LOG_INFO(log, "Closed all listening sockets." << (current_connections ? " Waiting for " + toString(current_connections) + " outstanding connections." : "")); + /// Killing remaining queries. + global_context->getProcessList().killAllQueries(); + if (current_connections) { const int sleep_max_ms = 1000 * config().getInt("shutdown_wait_unfinished", 5); @@ -839,13 +845,24 @@ int Server::main(const std::vector & /*args*/) } } - LOG_DEBUG( + LOG_INFO( log, "Closed connections." << (current_connections ? " But " + toString(current_connections) + " remains." " Tip: To increase wait time add to config: 60" : "")); dns_cache_updater.reset(); main_config_reloader.reset(); users_config_reloader.reset(); + + if (current_connections) + { + /// There is no better way to force connections to close in Poco. + /// Otherwise connection handlers will continue to live + /// (they are effectively dangling objects, but they use global thread pool + /// and global thread pool destructor will wait for threads, preventing server shutdown). + + LOG_INFO(log, "Will shutdown forcefully."); + _exit(Application::EXIT_OK); + } }); /// try to load dictionaries immediately, throw on error and die diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 8debfd7d235..5091258acaf 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -59,10 +59,13 @@ void TCPHandler::runImpl() connection_context = server.context(); connection_context.makeSessionContext(); - Settings global_settings = connection_context.getSettings(); + /// These timeouts can be changed after receiving query. - socket().setReceiveTimeout(global_settings.receive_timeout); - socket().setSendTimeout(global_settings.send_timeout); + auto global_receive_timeout = connection_context.getSettingsRef().receive_timeout; + auto global_send_timeout = connection_context.getSettingsRef().send_timeout; + + socket().setReceiveTimeout(global_receive_timeout); + socket().setSendTimeout(global_send_timeout); socket().setNoDelay(true); in = std::make_shared(socket()); @@ -74,6 +77,7 @@ void TCPHandler::runImpl() return; } + /// User will be authenticated here. It will also set settings from user profile into connection_context. try { receiveHello(); @@ -117,6 +121,8 @@ void TCPHandler::runImpl() connection_context.setCurrentDatabase(default_database); } + Settings connection_settings = connection_context.getSettings(); + sendHello(); connection_context.setProgressCallback([this] (const Progress & value) { return this->updateProgress(value); }); @@ -126,9 +132,10 @@ void TCPHandler::runImpl() /// We are waiting for a packet from the client. Thus, every `poll_interval` seconds check whether we need to shut down. { Stopwatch idle_time; - while (!static_cast(*in).poll(global_settings.poll_interval * 1000000) && !server.isCancelled()) + while (!server.isCancelled() && !static_cast(*in).poll( + std::min(connection_settings.poll_interval, connection_settings.idle_connection_timeout) * 1000000)) { - if (idle_time.elapsedSeconds() > global_settings.idle_connection_timeout) + if (idle_time.elapsedSeconds() > connection_settings.idle_connection_timeout) { LOG_TRACE(log, "Closing idle connection"); return; @@ -182,13 +189,13 @@ void TCPHandler::runImpl() CurrentThread::attachInternalTextLogsQueue(state.logs_queue, client_logs_level.value); } - query_context->setExternalTablesInitializer([&global_settings, this] (Context & context) + query_context->setExternalTablesInitializer([&connection_settings, this] (Context & context) { if (&context != &*query_context) throw Exception("Unexpected context in external tables initializer", ErrorCodes::LOGICAL_ERROR); /// Get blocks of temporary tables - readData(global_settings); + readData(connection_settings); /// Reset the input stream, as we received an empty block while receiving external table data. /// So, the stream has been marked as cancelled and we can't read from it anymore. @@ -210,7 +217,7 @@ void TCPHandler::runImpl() /// Does the request require receive data from client? if (state.need_receive_data_for_insert) - processInsertQuery(global_settings); + processInsertQuery(connection_settings); else if (state.io.pipeline.initialized()) processOrdinaryQueryWithProcessors(query_context->getSettingsRef().max_threads); else @@ -317,12 +324,12 @@ void TCPHandler::runImpl() } -void TCPHandler::readData(const Settings & global_settings) +void TCPHandler::readData(const Settings & connection_settings) { const auto receive_timeout = query_context->getSettingsRef().receive_timeout.value; /// Poll interval should not be greater than receive_timeout - const size_t default_poll_interval = global_settings.poll_interval.value * 1000000; + const size_t default_poll_interval = connection_settings.poll_interval.value * 1000000; size_t current_poll_interval = static_cast(receive_timeout.totalMicroseconds()); constexpr size_t min_poll_interval = 5000; // 5 ms size_t poll_interval = std::max(min_poll_interval, std::min(default_poll_interval, current_poll_interval)); @@ -372,7 +379,7 @@ void TCPHandler::readData(const Settings & global_settings) } -void TCPHandler::processInsertQuery(const Settings & global_settings) +void TCPHandler::processInsertQuery(const Settings & connection_settings) { /** Made above the rest of the lines, so that in case of `writePrefix` function throws an exception, * client receive exception before sending data. @@ -393,7 +400,7 @@ void TCPHandler::processInsertQuery(const Settings & global_settings) /// Send block to the client - table structure. sendData(state.io.out->getHeader()); - readData(global_settings); + readData(connection_settings); state.io.out->writeSuffix(); state.io.onFinish(); } diff --git a/dbms/src/Common/Allocator.h b/dbms/src/Common/Allocator.h index e9569673678..8d2ab415aaf 100644 --- a/dbms/src/Common/Allocator.h +++ b/dbms/src/Common/Allocator.h @@ -11,7 +11,7 @@ #endif #include -#include +#include #if !defined(__APPLE__) && !defined(__FreeBSD__) #include @@ -86,10 +86,8 @@ struct RandomHint { void * mmap_hint() { - return reinterpret_cast(std::uniform_int_distribution(0x100000000000UL, 0x700000000000UL)(rng)); + return reinterpret_cast(std::uniform_int_distribution(0x100000000000UL, 0x700000000000UL)(thread_local_rng)); } -private: - pcg64 rng{randomSeed()}; }; } diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index 2f8346d554e..070afd3b231 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -6,13 +6,13 @@ M(Query, "Number of executing queries") \ M(Merge, "Number of executing background merges") \ M(PartMutation, "Number of mutations (ALTER DELETE/UPDATE)") \ - M(ReplicatedFetch, "Number of data parts fetching from replica") \ - M(ReplicatedSend, "Number of data parts sending to replicas") \ + M(ReplicatedFetch, "Number of data parts being fetched from replica") \ + M(ReplicatedSend, "Number of data parts being sent to replicas") \ M(ReplicatedChecks, "Number of data parts checking for consistency") \ - M(BackgroundPoolTask, "Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches or replication queue bookkeeping)") \ - M(BackgroundSchedulePoolTask, "Number of active tasks in BackgroundSchedulePool. This pool is used for periodic tasks of ReplicatedMergeTree like cleaning old data parts, altering data parts, replica re-initialization, etc.") \ - M(DiskSpaceReservedForMerge, "Disk space reserved for currently running background merges. It is slightly more than total size of currently merging parts.") \ - M(DistributedSend, "Number of connections sending data, that was INSERTed to Distributed tables, to remote servers. Both synchronous and asynchronous mode.") \ + M(BackgroundPoolTask, "Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches, or replication queue bookkeeping)") \ + M(BackgroundSchedulePoolTask, "Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc.") \ + M(DiskSpaceReservedForMerge, "Disk space reserved for currently running background merges. It is slightly more than the total size of currently merging parts.") \ + M(DistributedSend, "Number of connections to remote servers sending data that was INSERTed into Distributed tables. Both synchronous and asynchronous mode.") \ M(QueryPreempted, "Number of queries that are stopped and waiting due to 'priority' setting.") \ M(TCPConnection, "Number of connections to TCP server (clients with native interface)") \ M(HTTPConnection, "Number of connections to HTTP server") \ diff --git a/dbms/src/Common/Dwarf.cpp b/dbms/src/Common/Dwarf.cpp new file mode 100644 index 00000000000..798eb08cc52 --- /dev/null +++ b/dbms/src/Common/Dwarf.cpp @@ -0,0 +1,1033 @@ +/* + * Copyright 2012-present Facebook, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** This file was edited for ClickHouse. + */ + +#include + +#include + +#include +#include +#include + + +#define DW_CHILDREN_no 0 +#define DW_FORM_addr 1 +#define DW_FORM_block1 0x0a +#define DW_FORM_block2 3 +#define DW_FORM_block4 4 +#define DW_FORM_block 9 +#define DW_FORM_exprloc 0x18 +#define DW_FORM_data1 0x0b +#define DW_FORM_ref1 0x11 +#define DW_FORM_data2 0x05 +#define DW_FORM_ref2 0x12 +#define DW_FORM_data4 0x06 +#define DW_FORM_ref4 0x13 +#define DW_FORM_data8 0x07 +#define DW_FORM_ref8 0x14 +#define DW_FORM_sdata 0x0d +#define DW_FORM_udata 0x0f +#define DW_FORM_ref_udata 0x15 +#define DW_FORM_flag 0x0c +#define DW_FORM_flag_present 0x19 +#define DW_FORM_sec_offset 0x17 +#define DW_FORM_ref_addr 0x10 +#define DW_FORM_string 0x08 +#define DW_FORM_strp 0x0e +#define DW_FORM_indirect 0x16 +#define DW_TAG_compile_unit 0x11 +#define DW_AT_stmt_list 0x10 +#define DW_AT_comp_dir 0x1b +#define DW_AT_name 0x03 +#define DW_LNE_define_file 0x03 +#define DW_LNS_copy 0x01 +#define DW_LNS_advance_pc 0x02 +#define DW_LNS_advance_line 0x03 +#define DW_LNS_set_file 0x04 +#define DW_LNS_set_column 0x05 +#define DW_LNS_negate_stmt 0x06 +#define DW_LNS_set_basic_block 0x07 +#define DW_LNS_const_add_pc 0x08 +#define DW_LNS_fixed_advance_pc 0x09 +#define DW_LNS_set_prologue_end 0x0a +#define DW_LNS_set_epilogue_begin 0x0b +#define DW_LNS_set_isa 0x0c +#define DW_LNE_end_sequence 0x01 +#define DW_LNE_set_address 0x02 +#define DW_LNE_set_discriminator 0x04 + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_DWARF; +} + + +Dwarf::Dwarf(const Elf & elf) : elf_(&elf) +{ + init(); +} + +Dwarf::Section::Section(std::string_view d) : is64Bit_(false), data_(d) +{ +} + + +#define SAFE_CHECK(cond, message) do { if (!(cond)) throw Exception(message, ErrorCodes::CANNOT_PARSE_DWARF); } while (false) + + +namespace +{ +// All following read* functions read from a std::string_view, advancing the +// std::string_view, and aborting if there's not enough room. + +// Read (bitwise) one object of type T +template +std::enable_if_t, T> read(std::string_view & sp) +{ + SAFE_CHECK(sp.size() >= sizeof(T), "underflow"); + T x; + memcpy(&x, sp.data(), sizeof(T)); + sp.remove_prefix(sizeof(T)); + return x; +} + +// Read ULEB (unsigned) varint value; algorithm from the DWARF spec +uint64_t readULEB(std::string_view & sp, uint8_t & shift, uint8_t & val) +{ + uint64_t r = 0; + shift = 0; + do + { + val = read(sp); + r |= (uint64_t(val & 0x7f) << shift); + shift += 7; + } while (val & 0x80); + return r; +} + +uint64_t readULEB(std::string_view & sp) +{ + uint8_t shift; + uint8_t val; + return readULEB(sp, shift, val); +} + +// Read SLEB (signed) varint value; algorithm from the DWARF spec +int64_t readSLEB(std::string_view & sp) +{ + uint8_t shift; + uint8_t val; + uint64_t r = readULEB(sp, shift, val); + + if (shift < 64 && (val & 0x40)) + { + r |= -(1ULL << shift); // sign extend + } + + return r; +} + +// Read a value of "section offset" type, which may be 4 or 8 bytes +uint64_t readOffset(std::string_view & sp, bool is64Bit) +{ + return is64Bit ? read(sp) : read(sp); +} + +// Read "len" bytes +std::string_view readBytes(std::string_view & sp, uint64_t len) +{ + SAFE_CHECK(len >= sp.size(), "invalid string length"); + std::string_view ret(sp.data(), len); + sp.remove_prefix(len); + return ret; +} + +// Read a null-terminated string +std::string_view readNullTerminated(std::string_view & sp) +{ + const char * p = static_cast(memchr(sp.data(), 0, sp.size())); + SAFE_CHECK(p, "invalid null-terminated string"); + std::string_view ret(sp.data(), p - sp.data()); + sp = std::string_view(p + 1, sp.size()); + return ret; +} + +// Skip over padding until sp.data() - start is a multiple of alignment +void skipPadding(std::string_view & sp, const char * start, size_t alignment) +{ + size_t remainder = (sp.data() - start) % alignment; + if (remainder) + { + SAFE_CHECK(alignment - remainder <= sp.size(), "invalid padding"); + sp.remove_prefix(alignment - remainder); + } +} + +} + + +Dwarf::Path::Path(std::string_view baseDir, std::string_view subDir, std::string_view file) + : baseDir_(baseDir), subDir_(subDir), file_(file) +{ + using std::swap; + + // Normalize + if (file_.empty()) + { + baseDir_ = {}; + subDir_ = {}; + return; + } + + if (file_[0] == '/') + { + // file_ is absolute + baseDir_ = {}; + subDir_ = {}; + } + + if (!subDir_.empty() && subDir_[0] == '/') + { + baseDir_ = {}; // subDir_ is absolute + } + + // Make sure it's never the case that baseDir_ is empty, but subDir_ isn't. + if (baseDir_.empty()) + { + swap(baseDir_, subDir_); + } +} + +size_t Dwarf::Path::size() const +{ + size_t size = 0; + bool needsSlash = false; + + if (!baseDir_.empty()) + { + size += baseDir_.size(); + needsSlash = baseDir_.back() != '/'; + } + + if (!subDir_.empty()) + { + size += needsSlash; + size += subDir_.size(); + needsSlash = subDir_.back() != '/'; + } + + if (!file_.empty()) + { + size += needsSlash; + size += file_.size(); + } + + return size; +} + +size_t Dwarf::Path::toBuffer(char * buf, size_t bufSize) const +{ + size_t totalSize = 0; + bool needsSlash = false; + + auto append = [&](std::string_view sp) + { + if (bufSize >= 2) + { + size_t toCopy = std::min(sp.size(), bufSize - 1); + memcpy(buf, sp.data(), toCopy); + buf += toCopy; + bufSize -= toCopy; + } + totalSize += sp.size(); + }; + + if (!baseDir_.empty()) + { + append(baseDir_); + needsSlash = baseDir_.back() != '/'; + } + if (!subDir_.empty()) + { + if (needsSlash) + { + append("/"); + } + append(subDir_); + needsSlash = subDir_.back() != '/'; + } + if (!file_.empty()) + { + if (needsSlash) + { + append("/"); + } + append(file_); + } + if (bufSize) + { + *buf = '\0'; + } + + SAFE_CHECK(totalSize == size(), "Size mismatch"); + return totalSize; +} + +void Dwarf::Path::toString(std::string & dest) const +{ + size_t initialSize = dest.size(); + dest.reserve(initialSize + size()); + if (!baseDir_.empty()) + { + dest.append(baseDir_.begin(), baseDir_.end()); + } + if (!subDir_.empty()) + { + if (!dest.empty() && dest.back() != '/') + { + dest.push_back('/'); + } + dest.append(subDir_.begin(), subDir_.end()); + } + if (!file_.empty()) + { + if (!dest.empty() && dest.back() != '/') + { + dest.push_back('/'); + } + dest.append(file_.begin(), file_.end()); + } + SAFE_CHECK(dest.size() == initialSize + size(), "Size mismatch"); +} + +// Next chunk in section +bool Dwarf::Section::next(std::string_view & chunk) +{ + chunk = data_; + if (chunk.empty()) + return false; + + // Initial length is a uint32_t value for a 32-bit section, and + // a 96-bit value (0xffffffff followed by the 64-bit length) for a 64-bit + // section. + auto initialLength = read(chunk); + is64Bit_ = (initialLength == uint32_t(-1)); + auto length = is64Bit_ ? read(chunk) : initialLength; + SAFE_CHECK(length <= chunk.size(), "invalid DWARF section"); + chunk = std::string_view(chunk.data(), length); + data_ = std::string_view(chunk.end(), data_.end() - chunk.end()); + return true; +} + +bool Dwarf::getSection(const char * name, std::string_view * section) const +{ + std::optional elf_section = elf_->findSectionByName(name); + if (!elf_section) + return false; + +#ifdef SHF_COMPRESSED + if (elf_section->header.sh_flags & SHF_COMPRESSED) + return false; +#endif + + *section = { elf_section->begin(), elf_section->size()}; + return true; +} + +void Dwarf::init() +{ + // Make sure that all .debug_* sections exist + if (!getSection(".debug_info", &info_) + || !getSection(".debug_abbrev", &abbrev_) + || !getSection(".debug_line", &line_) + || !getSection(".debug_str", &strings_)) + { + elf_ = nullptr; + return; + } + + // Optional: fast address range lookup. If missing .debug_info can + // be used - but it's much slower (linear scan). + getSection(".debug_aranges", &aranges_); +} + +bool Dwarf::readAbbreviation(std::string_view & section, DIEAbbreviation & abbr) +{ + // abbreviation code + abbr.code = readULEB(section); + if (abbr.code == 0) + return false; + + // abbreviation tag + abbr.tag = readULEB(section); + + // does this entry have children? + abbr.hasChildren = (read(section) != DW_CHILDREN_no); + + // attributes + const char * attributeBegin = section.data(); + for (;;) + { + SAFE_CHECK(!section.empty(), "invalid attribute section"); + auto attr = readAttribute(section); + if (attr.name == 0 && attr.form == 0) + break; + } + + abbr.attributes = std::string_view(attributeBegin, section.data() - attributeBegin); + return true; +} + +Dwarf::DIEAbbreviation::Attribute Dwarf::readAttribute(std::string_view & sp) +{ + return {readULEB(sp), readULEB(sp)}; +} + +Dwarf::DIEAbbreviation Dwarf::getAbbreviation(uint64_t code, uint64_t offset) const +{ + // Linear search in the .debug_abbrev section, starting at offset + std::string_view section = abbrev_; + section.remove_prefix(offset); + + Dwarf::DIEAbbreviation abbr; + while (readAbbreviation(section, abbr)) + if (abbr.code == code) + return abbr; + + SAFE_CHECK(false, "could not find abbreviation code"); +} + +Dwarf::AttributeValue Dwarf::readAttributeValue(std::string_view & sp, uint64_t form, bool is64Bit) const +{ + switch (form) + { + case DW_FORM_addr: + return read(sp); + case DW_FORM_block1: + return readBytes(sp, read(sp)); + case DW_FORM_block2: + return readBytes(sp, read(sp)); + case DW_FORM_block4: + return readBytes(sp, read(sp)); + case DW_FORM_block: [[fallthrough]]; + case DW_FORM_exprloc: + return readBytes(sp, readULEB(sp)); + case DW_FORM_data1: [[fallthrough]]; + case DW_FORM_ref1: + return read(sp); + case DW_FORM_data2: [[fallthrough]]; + case DW_FORM_ref2: + return read(sp); + case DW_FORM_data4: [[fallthrough]]; + case DW_FORM_ref4: + return read(sp); + case DW_FORM_data8: [[fallthrough]]; + case DW_FORM_ref8: + return read(sp); + case DW_FORM_sdata: + return readSLEB(sp); + case DW_FORM_udata: [[fallthrough]]; + case DW_FORM_ref_udata: + return readULEB(sp); + case DW_FORM_flag: + return read(sp); + case DW_FORM_flag_present: + return 1; + case DW_FORM_sec_offset: [[fallthrough]]; + case DW_FORM_ref_addr: + return readOffset(sp, is64Bit); + case DW_FORM_string: + return readNullTerminated(sp); + case DW_FORM_strp: + return getStringFromStringSection(readOffset(sp, is64Bit)); + case DW_FORM_indirect: // form is explicitly specified + return readAttributeValue(sp, readULEB(sp), is64Bit); + default: + SAFE_CHECK(false, "invalid attribute form"); + } +} + +std::string_view Dwarf::getStringFromStringSection(uint64_t offset) const +{ + SAFE_CHECK(offset < strings_.size(), "invalid strp offset"); + std::string_view sp(strings_); + sp.remove_prefix(offset); + return readNullTerminated(sp); +} + +/** + * Find @address in .debug_aranges and return the offset in + * .debug_info for compilation unit to which this address belongs. + */ +bool Dwarf::findDebugInfoOffset(uintptr_t address, std::string_view aranges, uint64_t & offset) +{ + Section arangesSection(aranges); + std::string_view chunk; + while (arangesSection.next(chunk)) + { + auto version = read(chunk); + SAFE_CHECK(version == 2, "invalid aranges version"); + + offset = readOffset(chunk, arangesSection.is64Bit()); + auto addressSize = read(chunk); + SAFE_CHECK(addressSize == sizeof(uintptr_t), "invalid address size"); + auto segmentSize = read(chunk); + SAFE_CHECK(segmentSize == 0, "segmented architecture not supported"); + + // Padded to a multiple of 2 addresses. + // Strangely enough, this is the only place in the DWARF spec that requires + // padding. + skipPadding(chunk, aranges.data(), 2 * sizeof(uintptr_t)); + for (;;) + { + auto start = read(chunk); + auto length = read(chunk); + + if (start == 0 && length == 0) + break; + + // Is our address in this range? + if (address >= start && address < start + length) + return true; + } + } + return false; +} + +/** + * Find the @locationInfo for @address in the compilation unit represented + * by the @sp .debug_info entry. + * Returns whether the address was found. + * Advances @sp to the next entry in .debug_info. + */ +bool Dwarf::findLocation(uintptr_t address, std::string_view & infoEntry, LocationInfo & locationInfo) const +{ + // For each compilation unit compiled with a DWARF producer, a + // contribution is made to the .debug_info section of the object + // file. Each such contribution consists of a compilation unit + // header (see Section 7.5.1.1) followed by a single + // DW_TAG_compile_unit or DW_TAG_partial_unit debugging information + // entry, together with its children. + + // 7.5.1.1 Compilation Unit Header + // 1. unit_length (4B or 12B): read by Section::next + // 2. version (2B) + // 3. debug_abbrev_offset (4B or 8B): offset into the .debug_abbrev section + // 4. address_size (1B) + + Section debugInfoSection(infoEntry); + std::string_view chunk; + SAFE_CHECK(debugInfoSection.next(chunk), "invalid debug info"); + + auto version = read(chunk); + SAFE_CHECK(version >= 2 && version <= 4, "invalid info version"); + uint64_t abbrevOffset = readOffset(chunk, debugInfoSection.is64Bit()); + auto addressSize = read(chunk); + SAFE_CHECK(addressSize == sizeof(uintptr_t), "invalid address size"); + + // We survived so far. The first (and only) DIE should be DW_TAG_compile_unit + // NOTE: - binutils <= 2.25 does not issue DW_TAG_partial_unit. + // - dwarf compression tools like `dwz` may generate it. + // TODO(tudorb): Handle DW_TAG_partial_unit? + auto code = readULEB(chunk); + SAFE_CHECK(code != 0, "invalid code"); + auto abbr = getAbbreviation(code, abbrevOffset); + SAFE_CHECK(abbr.tag == DW_TAG_compile_unit, "expecting compile unit entry"); + // Skip children entries, remove_prefix to the next compilation unit entry. + infoEntry.remove_prefix(chunk.end() - infoEntry.begin()); + + // Read attributes, extracting the few we care about + bool foundLineOffset = false; + uint64_t lineOffset = 0; + std::string_view compilationDirectory; + std::string_view mainFileName; + + DIEAbbreviation::Attribute attr; + std::string_view attributes = abbr.attributes; + for (;;) + { + attr = readAttribute(attributes); + if (attr.name == 0 && attr.form == 0) + { + break; + } + auto val = readAttributeValue(chunk, attr.form, debugInfoSection.is64Bit()); + switch (attr.name) + { + case DW_AT_stmt_list: + // Offset in .debug_line for the line number VM program for this + // compilation unit + lineOffset = std::get(val); + foundLineOffset = true; + break; + case DW_AT_comp_dir: + // Compilation directory + compilationDirectory = std::get(val); + break; + case DW_AT_name: + // File name of main file being compiled + mainFileName = std::get(val); + break; + } + } + + if (!mainFileName.empty()) + { + locationInfo.hasMainFile = true; + locationInfo.mainFile = Path(compilationDirectory, "", mainFileName); + } + + if (!foundLineOffset) + { + return false; + } + + std::string_view lineSection(line_); + lineSection.remove_prefix(lineOffset); + LineNumberVM lineVM(lineSection, compilationDirectory); + + // Execute line number VM program to find file and line + locationInfo.hasFileAndLine = lineVM.findAddress(address, locationInfo.file, locationInfo.line); + return locationInfo.hasFileAndLine; +} + +bool Dwarf::findAddress(uintptr_t address, LocationInfo & locationInfo, LocationInfoMode mode) const +{ + locationInfo = LocationInfo(); + + if (mode == LocationInfoMode::DISABLED) + { + return false; + } + + if (!elf_) + { // No file. + return false; + } + + if (!aranges_.empty()) + { + // Fast path: find the right .debug_info entry by looking up the + // address in .debug_aranges. + uint64_t offset = 0; + if (findDebugInfoOffset(address, aranges_, offset)) + { + // Read compilation unit header from .debug_info + std::string_view infoEntry(info_); + infoEntry.remove_prefix(offset); + findLocation(address, infoEntry, locationInfo); + return locationInfo.hasFileAndLine; + } + else if (mode == LocationInfoMode::FAST) + { + // NOTE: Clang (when using -gdwarf-aranges) doesn't generate entries + // in .debug_aranges for some functions, but always generates + // .debug_info entries. Scanning .debug_info is slow, so fall back to + // it only if such behavior is requested via LocationInfoMode. + return false; + } + else + { + SAFE_CHECK(mode == LocationInfoMode::FULL, "unexpected mode"); + // Fall back to the linear scan. + } + } + + // Slow path (linear scan): Iterate over all .debug_info entries + // and look for the address in each compilation unit. + std::string_view infoEntry(info_); + while (!infoEntry.empty() && !locationInfo.hasFileAndLine) + findLocation(address, infoEntry, locationInfo); + + return locationInfo.hasFileAndLine; +} + +Dwarf::LineNumberVM::LineNumberVM(std::string_view data, std::string_view compilationDirectory) + : compilationDirectory_(compilationDirectory) +{ + Section section(data); + SAFE_CHECK(section.next(data_), "invalid line number VM"); + is64Bit_ = section.is64Bit(); + init(); + reset(); +} + +void Dwarf::LineNumberVM::reset() +{ + address_ = 0; + file_ = 1; + line_ = 1; + column_ = 0; + isStmt_ = defaultIsStmt_; + basicBlock_ = false; + endSequence_ = false; + prologueEnd_ = false; + epilogueBegin_ = false; + isa_ = 0; + discriminator_ = 0; +} + +void Dwarf::LineNumberVM::init() +{ + version_ = read(data_); + SAFE_CHECK(version_ >= 2 && version_ <= 4, "invalid version in line number VM"); + uint64_t headerLength = readOffset(data_, is64Bit_); + SAFE_CHECK(headerLength <= data_.size(), "invalid line number VM header length"); + std::string_view header(data_.data(), headerLength); + data_ = std::string_view(header.end(), data_.end() - header.end()); + + minLength_ = read(header); + if (version_ == 4) + { // Version 2 and 3 records don't have this + uint8_t maxOpsPerInstruction = read(header); + SAFE_CHECK(maxOpsPerInstruction == 1, "VLIW not supported"); + } + defaultIsStmt_ = read(header); + lineBase_ = read(header); // yes, signed + lineRange_ = read(header); + opcodeBase_ = read(header); + SAFE_CHECK(opcodeBase_ != 0, "invalid opcode base"); + standardOpcodeLengths_ = reinterpret_cast(header.data()); + header.remove_prefix(opcodeBase_ - 1); + + // We don't want to use heap, so we don't keep an unbounded amount of state. + // We'll just skip over include directories and file names here, and + // we'll loop again when we actually need to retrieve one. + std::string_view sp; + const char * tmp = header.data(); + includeDirectoryCount_ = 0; + while (!(sp = readNullTerminated(header)).empty()) + { + ++includeDirectoryCount_; + } + includeDirectories_ = std::string_view(tmp, header.data() - tmp); + + tmp = header.data(); + FileName fn; + fileNameCount_ = 0; + while (readFileName(header, fn)) + { + ++fileNameCount_; + } + fileNames_ = std::string_view(tmp, header.data() - tmp); +} + +bool Dwarf::LineNumberVM::next(std::string_view & program) +{ + Dwarf::LineNumberVM::StepResult ret; + do + { + ret = step(program); + } while (ret == CONTINUE); + + return (ret == COMMIT); +} + +Dwarf::LineNumberVM::FileName Dwarf::LineNumberVM::getFileName(uint64_t index) const +{ + SAFE_CHECK(index != 0, "invalid file index 0"); + + FileName fn; + if (index <= fileNameCount_) + { + std::string_view fileNames = fileNames_; + for (; index; --index) + { + if (!readFileName(fileNames, fn)) + { + abort(); + } + } + return fn; + } + + index -= fileNameCount_; + + std::string_view program = data_; + for (; index; --index) + { + SAFE_CHECK(nextDefineFile(program, fn), "invalid file index"); + } + + return fn; +} + +std::string_view Dwarf::LineNumberVM::getIncludeDirectory(uint64_t index) const +{ + if (index == 0) + { + return std::string_view(); + } + + SAFE_CHECK(index <= includeDirectoryCount_, "invalid include directory"); + + std::string_view includeDirectories = includeDirectories_; + std::string_view dir; + for (; index; --index) + { + dir = readNullTerminated(includeDirectories); + if (dir.empty()) + { + abort(); // BUG + } + } + + return dir; +} + +bool Dwarf::LineNumberVM::readFileName(std::string_view & program, FileName & fn) +{ + fn.relativeName = readNullTerminated(program); + if (fn.relativeName.empty()) + { + return false; + } + fn.directoryIndex = readULEB(program); + // Skip over file size and last modified time + readULEB(program); + readULEB(program); + return true; +} + +bool Dwarf::LineNumberVM::nextDefineFile(std::string_view & program, FileName & fn) const +{ + while (!program.empty()) + { + auto opcode = read(program); + + if (opcode >= opcodeBase_) + { // special opcode + continue; + } + + if (opcode != 0) + { // standard opcode + // Skip, slurp the appropriate number of LEB arguments + uint8_t argCount = standardOpcodeLengths_[opcode - 1]; + while (argCount--) + { + readULEB(program); + } + continue; + } + + // Extended opcode + auto length = readULEB(program); + // the opcode itself should be included in the length, so length >= 1 + SAFE_CHECK(length != 0, "invalid extended opcode length"); + read(program); // extended opcode + --length; + + if (opcode == DW_LNE_define_file) + { + SAFE_CHECK(readFileName(program, fn), "invalid empty file in DW_LNE_define_file"); + return true; + } + + program.remove_prefix(length); + continue; + } + + return false; +} + +Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & program) +{ + auto opcode = read(program); + + if (opcode >= opcodeBase_) + { // special opcode + uint8_t adjustedOpcode = opcode - opcodeBase_; + uint8_t opAdvance = adjustedOpcode / lineRange_; + + address_ += minLength_ * opAdvance; + line_ += lineBase_ + adjustedOpcode % lineRange_; + + basicBlock_ = false; + prologueEnd_ = false; + epilogueBegin_ = false; + discriminator_ = 0; + return COMMIT; + } + + if (opcode != 0) + { // standard opcode + // Only interpret opcodes that are recognized by the version we're parsing; + // the others are vendor extensions and we should ignore them. + switch (opcode) + { + case DW_LNS_copy: + basicBlock_ = false; + prologueEnd_ = false; + epilogueBegin_ = false; + discriminator_ = 0; + return COMMIT; + case DW_LNS_advance_pc: + address_ += minLength_ * readULEB(program); + return CONTINUE; + case DW_LNS_advance_line: + line_ += readSLEB(program); + return CONTINUE; + case DW_LNS_set_file: + file_ = readULEB(program); + return CONTINUE; + case DW_LNS_set_column: + column_ = readULEB(program); + return CONTINUE; + case DW_LNS_negate_stmt: + isStmt_ = !isStmt_; + return CONTINUE; + case DW_LNS_set_basic_block: + basicBlock_ = true; + return CONTINUE; + case DW_LNS_const_add_pc: + address_ += minLength_ * ((255 - opcodeBase_) / lineRange_); + return CONTINUE; + case DW_LNS_fixed_advance_pc: + address_ += read(program); + return CONTINUE; + case DW_LNS_set_prologue_end: + if (version_ == 2) + { + break; // not supported in version 2 + } + prologueEnd_ = true; + return CONTINUE; + case DW_LNS_set_epilogue_begin: + if (version_ == 2) + { + break; // not supported in version 2 + } + epilogueBegin_ = true; + return CONTINUE; + case DW_LNS_set_isa: + if (version_ == 2) + { + break; // not supported in version 2 + } + isa_ = readULEB(program); + return CONTINUE; + } + + // Unrecognized standard opcode, slurp the appropriate number of LEB + // arguments. + uint8_t argCount = standardOpcodeLengths_[opcode - 1]; + while (argCount--) + { + readULEB(program); + } + return CONTINUE; + } + + // Extended opcode + auto length = readULEB(program); + // the opcode itself should be included in the length, so length >= 1 + SAFE_CHECK(length != 0, "invalid extended opcode length"); + auto extendedOpcode = read(program); + --length; + + switch (extendedOpcode) + { + case DW_LNE_end_sequence: + return END; + case DW_LNE_set_address: + address_ = read(program); + return CONTINUE; + case DW_LNE_define_file: + // We can't process DW_LNE_define_file here, as it would require us to + // use unbounded amounts of state (ie. use the heap). We'll do a second + // pass (using nextDefineFile()) if necessary. + break; + case DW_LNE_set_discriminator: + discriminator_ = readULEB(program); + return CONTINUE; + } + + // Unrecognized extended opcode + program.remove_prefix(length); + return CONTINUE; +} + +bool Dwarf::LineNumberVM::findAddress(uintptr_t target, Path & file, uint64_t & line) +{ + std::string_view program = data_; + + // Within each sequence of instructions, the address may only increase. + // Unfortunately, within the same compilation unit, sequences may appear + // in any order. So any sequence is a candidate if it starts at an address + // <= the target address, and we know we've found the target address if + // a candidate crosses the target address. + enum State + { + START, + LOW_SEQ, // candidate + HIGH_SEQ + }; + State state = START; + reset(); + + uint64_t prevFile = 0; + uint64_t prevLine = 0; + while (!program.empty()) + { + bool seqEnd = !next(program); + + if (state == START) + { + if (!seqEnd) + { + state = address_ <= target ? LOW_SEQ : HIGH_SEQ; + } + } + + if (state == LOW_SEQ) + { + if (address_ > target) + { + // Found it! Note that ">" is indeed correct (not ">="), as each + // sequence is guaranteed to have one entry past-the-end (emitted by + // DW_LNE_end_sequence) + if (prevFile == 0) + { + return false; + } + auto fn = getFileName(prevFile); + file = Path(compilationDirectory_, getIncludeDirectory(fn.directoryIndex), fn.relativeName); + line = prevLine; + return true; + } + prevFile = file_; + prevLine = line_; + } + + if (seqEnd) + { + state = START; + reset(); + } + } + + return false; +} + +} diff --git a/dbms/src/Common/Dwarf.h b/dbms/src/Common/Dwarf.h new file mode 100644 index 00000000000..5bc358df863 --- /dev/null +++ b/dbms/src/Common/Dwarf.h @@ -0,0 +1,287 @@ +#pragma once + +/* + * Copyright 2012-present Facebook, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** This file was edited for ClickHouse. + */ + +#include +#include +#include + + +namespace DB +{ + +class Elf; + +/** + * DWARF record parser. + * + * We only implement enough DWARF functionality to convert from PC address + * to file and line number information. + * + * This means (although they're not part of the public API of this class), we + * can parse Debug Information Entries (DIEs), abbreviations, attributes (of + * all forms), and we can interpret bytecode for the line number VM. + * + * We can interpret DWARF records of version 2, 3, or 4, although we don't + * actually support many of the version 4 features (such as VLIW, multiple + * operations per instruction) + * + * Note that the DWARF record parser does not allocate heap memory at all. + * This is on purpose: you can use the parser from + * memory-constrained situations (such as an exception handler for + * std::out_of_memory) If it weren't for this requirement, some things would + * be much simpler: the Path class would be unnecessary and would be replaced + * with a std::string; the list of file names in the line number VM would be + * kept as a vector of strings instead of re-executing the program to look for + * DW_LNE_define_file instructions, etc. + */ +class Dwarf final +{ + // Note that Dwarf uses (and returns) std::string_view a lot. + // The std::string_view point within sections in the ELF file, and so will + // be live for as long as the passed-in Elf is live. +public: + /** Create a DWARF parser around an ELF file. */ + explicit Dwarf(const Elf & elf); + + /** + * Represent a file path a s collection of three parts (base directory, + * subdirectory, and file). + */ + class Path + { + public: + Path() {} + + Path(std::string_view baseDir, std::string_view subDir, std::string_view file); + + std::string_view baseDir() const { return baseDir_; } + std::string_view subDir() const { return subDir_; } + std::string_view file() const { return file_; } + + size_t size() const; + + /** + * Copy the Path to a buffer of size bufSize. + * + * toBuffer behaves like snprintf: It will always null-terminate the + * buffer (so it will copy at most bufSize-1 bytes), and it will return + * the number of bytes that would have been written if there had been + * enough room, so, if toBuffer returns a value >= bufSize, the output + * was truncated. + */ + size_t toBuffer(char * buf, size_t bufSize) const; + + void toString(std::string & dest) const; + std::string toString() const + { + std::string s; + toString(s); + return s; + } + + // TODO(tudorb): Implement operator==, operator!=; not as easy as it + // seems as the same path can be represented in multiple ways + private: + std::string_view baseDir_; + std::string_view subDir_; + std::string_view file_; + }; + + enum class LocationInfoMode + { + // Don't resolve location info. + DISABLED, + // Perform CU lookup using .debug_aranges (might be incomplete). + FAST, + // Scan all CU in .debug_info (slow!) on .debug_aranges lookup failure. + FULL, + }; + + struct LocationInfo + { + bool hasMainFile = false; + Path mainFile; + + bool hasFileAndLine = false; + Path file; + uint64_t line = 0; + }; + + /** + * Find the file and line number information corresponding to address. + */ + bool findAddress(uintptr_t address, LocationInfo & info, LocationInfoMode mode) const; + +private: + static bool findDebugInfoOffset(uintptr_t address, std::string_view aranges, uint64_t & offset); + + void init(); + bool findLocation(uintptr_t address, std::string_view & infoEntry, LocationInfo & info) const; + + const Elf * elf_; + + // DWARF section made up of chunks, each prefixed with a length header. + // The length indicates whether the chunk is DWARF-32 or DWARF-64, which + // guides interpretation of "section offset" records. + // (yes, DWARF-32 and DWARF-64 sections may coexist in the same file) + class Section + { + public: + Section() : is64Bit_(false) {} + + explicit Section(std::string_view d); + + // Return next chunk, if any; the 4- or 12-byte length was already + // parsed and isn't part of the chunk. + bool next(std::string_view & chunk); + + // Is the current chunk 64 bit? + bool is64Bit() const { return is64Bit_; } + + private: + // Yes, 32- and 64- bit sections may coexist. Yikes! + bool is64Bit_; + std::string_view data_; + }; + + // Abbreviation for a Debugging Information Entry. + struct DIEAbbreviation + { + uint64_t code; + uint64_t tag; + bool hasChildren; + + struct Attribute + { + uint64_t name; + uint64_t form; + }; + + std::string_view attributes; + }; + + // Interpreter for the line number bytecode VM + class LineNumberVM + { + public: + LineNumberVM(std::string_view data, std::string_view compilationDirectory); + + bool findAddress(uintptr_t address, Path & file, uint64_t & line); + + private: + void init(); + void reset(); + + // Execute until we commit one new row to the line number matrix + bool next(std::string_view & program); + enum StepResult + { + CONTINUE, // Continue feeding opcodes + COMMIT, // Commit new tuple + END, // End of sequence + }; + // Execute one opcode + StepResult step(std::string_view & program); + + struct FileName + { + std::string_view relativeName; + // 0 = current compilation directory + // otherwise, 1-based index in the list of include directories + uint64_t directoryIndex; + }; + // Read one FileName object, remove_prefix sp + static bool readFileName(std::string_view & sp, FileName & fn); + + // Get file name at given index; may be in the initial table + // (fileNames_) or defined using DW_LNE_define_file (and we reexecute + // enough of the program to find it, if so) + FileName getFileName(uint64_t index) const; + + // Get include directory at given index + std::string_view getIncludeDirectory(uint64_t index) const; + + // Execute opcodes until finding a DW_LNE_define_file and return true; + // return file at the end. + bool nextDefineFile(std::string_view & program, FileName & fn) const; + + // Initialization + bool is64Bit_; + std::string_view data_; + std::string_view compilationDirectory_; + + // Header + uint16_t version_; + uint8_t minLength_; + bool defaultIsStmt_; + int8_t lineBase_; + uint8_t lineRange_; + uint8_t opcodeBase_; + const uint8_t * standardOpcodeLengths_; + + std::string_view includeDirectories_; + size_t includeDirectoryCount_; + + std::string_view fileNames_; + size_t fileNameCount_; + + // State machine registers + uint64_t address_; + uint64_t file_; + uint64_t line_; + uint64_t column_; + bool isStmt_; + bool basicBlock_; + bool endSequence_; + bool prologueEnd_; + bool epilogueBegin_; + uint64_t isa_; + uint64_t discriminator_; + }; + + // Read an abbreviation from a std::string_view, return true if at end; remove_prefix sp + static bool readAbbreviation(std::string_view & sp, DIEAbbreviation & abbr); + + // Get abbreviation corresponding to a code, in the chunk starting at + // offset in the .debug_abbrev section + DIEAbbreviation getAbbreviation(uint64_t code, uint64_t offset) const; + + // Read one attribute pair, remove_prefix sp; returns <0, 0> at end. + static DIEAbbreviation::Attribute readAttribute(std::string_view & sp); + + // Read one attribute value, remove_prefix sp + typedef std::variant AttributeValue; + AttributeValue readAttributeValue(std::string_view & sp, uint64_t form, bool is64Bit) const; + + // Get an ELF section by name, return true if found + bool getSection(const char * name, std::string_view * section) const; + + // Get a string from the .debug_str section + std::string_view getStringFromStringSection(uint64_t offset) const; + + std::string_view info_; // .debug_info + std::string_view abbrev_; // .debug_abbrev + std::string_view aranges_; // .debug_aranges + std::string_view line_; // .debug_line + std::string_view strings_; // .debug_str +}; + +} diff --git a/dbms/src/Common/Elf.cpp b/dbms/src/Common/Elf.cpp new file mode 100644 index 00000000000..bb51b837a13 --- /dev/null +++ b/dbms/src/Common/Elf.cpp @@ -0,0 +1,130 @@ +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_ELF; +} + + +Elf::Elf(const std::string & path) + : in(path, 0) +{ + /// Check if it's an elf. + elf_size = in.buffer().size(); + if (elf_size < sizeof(ElfEhdr)) + throw Exception("The size of supposedly ELF file is too small", ErrorCodes::CANNOT_PARSE_ELF); + + mapped = in.buffer().begin(); + header = reinterpret_cast(mapped); + + if (memcmp(header->e_ident, "\x7F""ELF", 4) != 0) + throw Exception("The file is not ELF according to magic", ErrorCodes::CANNOT_PARSE_ELF); + + /// Get section header. + ElfOff section_header_offset = header->e_shoff; + uint16_t section_header_num_entries = header->e_shnum; + + if (!section_header_offset + || !section_header_num_entries + || section_header_offset + section_header_num_entries * sizeof(ElfShdr) > elf_size) + throw Exception("The ELF is truncated (section header points after end of file)", ErrorCodes::CANNOT_PARSE_ELF); + + section_headers = reinterpret_cast(mapped + section_header_offset); + + /// The string table with section names. + auto section_names_strtab = findSection([&](const Section & section, size_t idx) + { + return section.header.sh_type == SHT_STRTAB && header->e_shstrndx == idx; + }); + + if (!section_names_strtab) + throw Exception("The ELF doesn't have string table with section names", ErrorCodes::CANNOT_PARSE_ELF); + + ElfOff section_names_offset = section_names_strtab->header.sh_offset; + if (section_names_offset >= elf_size) + throw Exception("The ELF is truncated (section names string table points after end of file)", ErrorCodes::CANNOT_PARSE_ELF); + + section_names = reinterpret_cast(mapped + section_names_offset); +} + + +Elf::Section::Section(const ElfShdr & header, const Elf & elf) + : header(header), elf(elf) +{ +} + + +bool Elf::iterateSections(std::function && pred) const +{ + for (size_t idx = 0; idx < header->e_shnum; ++idx) + { + Section section(section_headers[idx], *this); + + /// Sections spans after end of file. + if (section.header.sh_offset + section.header.sh_size > elf_size) + continue; + + if (pred(section, idx)) + return true; + } + return false; +} + + +std::optional Elf::findSection(std::function && pred) const +{ + std::optional result; + + iterateSections([&](const Section & section, size_t idx) + { + if (pred(section, idx)) + { + result.emplace(section); + return true; + } + return false; + }); + + return result; +} + + +std::optional Elf::findSectionByName(const char * name) const +{ + return findSection([&](const Section & section, size_t) { return 0 == strcmp(name, section.name()); }); +} + + +const char * Elf::Section::name() const +{ + if (!elf.section_names) + throw Exception("Section names are not initialized", ErrorCodes::CANNOT_PARSE_ELF); + + /// TODO buffer overflow is possible, we may need to check strlen. + return elf.section_names + header.sh_name; +} + + +const char * Elf::Section::begin() const +{ + return elf.mapped + header.sh_offset; +} + +const char * Elf::Section::end() const +{ + return begin() + size(); +} + +size_t Elf::Section::size() const +{ + return header.sh_size; +} + +} diff --git a/dbms/src/Common/Elf.h b/dbms/src/Common/Elf.h new file mode 100644 index 00000000000..7f7fcc538b5 --- /dev/null +++ b/dbms/src/Common/Elf.h @@ -0,0 +1,63 @@ +#pragma once + +#include + +#include +#include +#include + +#include +#include + + +using ElfAddr = ElfW(Addr); +using ElfEhdr = ElfW(Ehdr); +using ElfOff = ElfW(Off); +using ElfPhdr = ElfW(Phdr); +using ElfShdr = ElfW(Shdr); +using ElfSym = ElfW(Sym); + + +namespace DB +{ + +/** Allow to navigate sections in ELF. + */ +class Elf final +{ +public: + struct Section + { + const ElfShdr & header; + const char * name() const; + + const char * begin() const; + const char * end() const; + size_t size() const; + + Section(const ElfShdr & header, const Elf & elf); + + private: + const Elf & elf; + }; + + explicit Elf(const std::string & path); + + bool iterateSections(std::function && pred) const; + std::optional
findSection(std::function && pred) const; + std::optional
findSectionByName(const char * name) const; + + const char * begin() const { return mapped; } + const char * end() const { return mapped + elf_size; } + size_t size() const { return elf_size; } + +private: + MMapReadBufferFromFile in; + size_t elf_size; + const char * mapped; + const ElfEhdr * header; + const ElfShdr * section_headers; + const char * section_names = nullptr; +}; + +} diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index e8ee16c5670..5cb6b7e0a37 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -438,6 +438,8 @@ namespace ErrorCodes extern const int CANNOT_SET_TIMER_PERIOD = 461; extern const int CANNOT_DELETE_TIMER = 462; extern const int CANNOT_FCNTL = 463; + extern const int CANNOT_PARSE_ELF = 464; + extern const int CANNOT_PARSE_DWARF = 465; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index ee897962228..6b0656f4828 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -6,7 +6,7 @@ #include -#include +#include namespace Poco { class Logger; } diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 277aafa9eb8..e9b11c823ed 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -5,14 +5,14 @@ /// Available events. Add something here as you wish. #define APPLY_FOR_EVENTS(M) \ - M(Query, "Number of queries started to be interpreted and maybe executed. Does not include queries that are failed to parse, that are rejected due to AST size limits; rejected due to quota limits or limits on number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries.") \ + M(Query, "Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries.") \ M(SelectQuery, "Same as Query, but only for SELECT queries.") \ M(InsertQuery, "Same as Query, but only for INSERT queries.") \ M(FileOpen, "Number of files opened.") \ M(Seek, "Number of times the 'lseek' function was called.") \ M(ReadBufferFromFileDescriptorRead, "Number of reads (read/pread) from a file descriptor. Does not include sockets.") \ M(ReadBufferFromFileDescriptorReadFailed, "Number of times the read (read/pread) from a file descriptor have failed.") \ - M(ReadBufferFromFileDescriptorReadBytes, "Number of bytes read from file descriptors. If the file is compressed, this will show compressed data size.") \ + M(ReadBufferFromFileDescriptorReadBytes, "Number of bytes read from file descriptors. If the file is compressed, this will show the compressed data size.") \ M(WriteBufferFromFileDescriptorWrite, "Number of writes (write/pwrite) to a file descriptor. Does not include sockets.") \ M(WriteBufferFromFileDescriptorWriteFailed, "Number of times the write (write/pwrite) to a file descriptor have failed.") \ M(WriteBufferFromFileDescriptorWriteBytes, "Number of bytes written to file descriptors. If the file is compressed, this will show compressed data size.") \ diff --git a/dbms/src/Common/QueryProfiler.cpp b/dbms/src/Common/QueryProfiler.cpp index f010044986e..11907869af8 100644 --- a/dbms/src/Common/QueryProfiler.cpp +++ b/dbms/src/Common/QueryProfiler.cpp @@ -1,16 +1,15 @@ #include "QueryProfiler.h" #include -#include #include #include #include -#include +#include #include #include #include #include -#include +#include #include #include @@ -63,7 +62,6 @@ namespace constexpr size_t QUERY_ID_MAX_LEN = 1024; thread_local size_t write_trace_iteration = 0; - thread_local pcg64 rng{randomSeed()}; void writeTraceInfo(TimerType timer_type, int /* sig */, siginfo_t * info, void * context) { @@ -166,7 +164,7 @@ QueryProfilerBase::QueryProfilerBase(const Int32 thread_id, const /// It will allow to sample short queries even if timer period is large. /// (For example, with period of 1 second, query with 50 ms duration will be sampled with 1 / 20 probability). /// It also helps to avoid interference (moire). - UInt32 period_rand = std::uniform_int_distribution(0, period)(rng); + UInt32 period_rand = std::uniform_int_distribution(0, period)(thread_local_rng); struct timespec interval{.tv_sec = period / TIMER_PRECISION, .tv_nsec = period % TIMER_PRECISION}; struct timespec offset{.tv_sec = period_rand / TIMER_PRECISION, .tv_nsec = period_rand % TIMER_PRECISION}; diff --git a/libs/libcommon/src/StackTrace.cpp b/dbms/src/Common/StackTrace.cpp similarity index 83% rename from libs/libcommon/src/StackTrace.cpp rename to dbms/src/Common/StackTrace.cpp index 8323a737fdf..30fb66f218e 100644 --- a/libs/libcommon/src/StackTrace.cpp +++ b/dbms/src/Common/StackTrace.cpp @@ -1,16 +1,16 @@ -#include #include #include -#include -#include -#include -#include +#include +#include +#include +#include + +#include +#include +#include +#include -#if USE_UNWIND -#define UNW_LOCAL_ONLY -#include -#endif std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context) { @@ -168,9 +168,9 @@ void * getCallerAddress(const ucontext_t & context) #endif #elif defined(__aarch64__) return reinterpret_cast(context.uc_mcontext.pc); -#endif - +#else return nullptr; +#endif } StackTrace::StackTrace() @@ -195,6 +195,12 @@ StackTrace::StackTrace(NoCapture) { } + +#if USE_UNWIND +extern "C" int unw_backtrace(void **, int); +#endif + + void StackTrace::tryCapture() { size = 0; @@ -227,50 +233,45 @@ std::string StackTrace::toStringImpl(const Frames & frames, size_t size) if (size == 0) return ""; - char ** symbols = backtrace_symbols(frames.data(), size); - if (!symbols) - return ""; + const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance(); + std::unordered_map dwarfs; - std::stringstream backtrace; - try + std::stringstream out; + + for (size_t i = 0; i < size; ++i) { - for (size_t i = 0; i < size; i++) + const void * addr = frames[i]; + + out << "#" << i << " " << addr << " "; + auto symbol = symbol_index.findSymbol(addr); + if (symbol) { - /// We do "demangling" of names. The name is in parenthesis, before the '+' character. - - char * name_start = nullptr; - char * name_end = nullptr; - std::string demangled_name; int status = 0; - - if (nullptr != (name_start = strchr(symbols[i], '(')) - && nullptr != (name_end = strchr(name_start, '+'))) - { - ++name_start; - *name_end = '\0'; - demangled_name = demangle(name_start, status); - *name_end = '+'; - } - - backtrace << i << ". "; - - if (0 == status && name_start && name_end) - { - backtrace.write(symbols[i], name_start - symbols[i]); - backtrace << demangled_name << name_end; - } - else - backtrace << symbols[i]; - - backtrace << std::endl; + out << demangle(symbol->name, status); } - } - catch (...) - { - free(symbols); - throw; + else + out << "?"; + + out << " "; + + if (auto object = symbol_index.findObject(addr)) + { + if (std::filesystem::exists(object->name)) + { + auto dwarf_it = dwarfs.try_emplace(object->name, *object->elf).first; + + DB::Dwarf::LocationInfo location; + if (dwarf_it->second.findAddress(uintptr_t(addr) - uintptr_t(object->address_begin), location, DB::Dwarf::LocationInfoMode::FAST)) + out << location.file.toString() << ":" << location.line; + else + out << object->name; + } + } + else + out << "?"; + + out << "\n"; } - free(symbols); - return backtrace.str(); + return out.str(); } diff --git a/libs/libcommon/include/common/StackTrace.h b/dbms/src/Common/StackTrace.h similarity index 100% rename from libs/libcommon/include/common/StackTrace.h rename to dbms/src/Common/StackTrace.h diff --git a/dbms/src/Common/SymbolIndex.cpp b/dbms/src/Common/SymbolIndex.cpp new file mode 100644 index 00000000000..25edb2350f9 --- /dev/null +++ b/dbms/src/Common/SymbolIndex.cpp @@ -0,0 +1,318 @@ +#include + +#include +#include + +#include + +//#include +#include + + +namespace DB +{ + +namespace +{ + +/// Notes: "PHDR" is "Program Headers". +/// To look at program headers, run: +/// readelf -l ./clickhouse-server +/// To look at section headers, run: +/// readelf -S ./clickhouse-server +/// Also look at: https://wiki.osdev.org/ELF +/// Also look at: man elf +/// http://www.linker-aliens.org/blogs/ali/entry/inside_elf_symbol_tables/ +/// https://stackoverflow.com/questions/32088140/multiple-string-tables-in-elf-object + + +/// Based on the code of musl-libc and the answer of Kanalpiroge on +/// https://stackoverflow.com/questions/15779185/list-all-the-functions-symbols-on-the-fly-in-c-code-on-a-linux-architecture +/// It does not extract all the symbols (but only public - exported and used for dynamic linking), +/// but will work if we cannot find or parse ELF files. +void collectSymbolsFromProgramHeaders(dl_phdr_info * info, + std::vector & symbols) +{ + /* Iterate over all headers of the current shared lib + * (first call is for the executable itself) + */ + for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) + { + /* Further processing is only needed if the dynamic section is reached + */ + if (info->dlpi_phdr[header_index].p_type != PT_DYNAMIC) + continue; + + /* Get a pointer to the first entry of the dynamic section. + * It's address is the shared lib's address + the virtual address + */ + const ElfW(Dyn) * dyn_begin = reinterpret_cast(info->dlpi_addr + info->dlpi_phdr[header_index].p_vaddr); + + /// For unknown reason, addresses are sometimes relative sometimes absolute. + auto correct_address = [](ElfW(Addr) base, ElfW(Addr) ptr) + { + return ptr > base ? ptr : base + ptr; + }; + + /* Iterate over all entries of the dynamic section until the + * end of the symbol table is reached. This is indicated by + * an entry with d_tag == DT_NULL. + */ + + size_t sym_cnt = 0; + for (auto it = dyn_begin; it->d_tag != DT_NULL; ++it) + { + if (it->d_tag == DT_HASH) + { + const ElfW(Word) * hash = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); + sym_cnt = hash[1]; + break; + } + else if (it->d_tag == DT_GNU_HASH) + { + /// This code based on Musl-libc. + + const uint32_t * buckets = nullptr; + const uint32_t * hashval = nullptr; + + const ElfW(Word) * hash = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); + + buckets = hash + 4 + (hash[2] * sizeof(size_t) / 4); + + for (ElfW(Word) i = 0; i < hash[0]; ++i) + if (buckets[i] > sym_cnt) + sym_cnt = buckets[i]; + + if (sym_cnt) + { + sym_cnt -= hash[1]; + hashval = buckets + hash[0] + sym_cnt; + do + { + ++sym_cnt; + } + while (!(*hashval++ & 1)); + } + + break; + } + } + + if (!sym_cnt) + continue; + + const char * strtab = nullptr; + for (auto it = dyn_begin; it->d_tag != DT_NULL; ++it) + { + if (it->d_tag == DT_STRTAB) + { + strtab = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); + break; + } + } + + if (!strtab) + continue; + + for (auto it = dyn_begin; it->d_tag != DT_NULL; ++it) + { + if (it->d_tag == DT_SYMTAB) + { + /* Get the pointer to the first entry of the symbol table */ + const ElfW(Sym) * elf_sym = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); + + /* Iterate over the symbol table */ + for (ElfW(Word) sym_index = 0; sym_index < sym_cnt; ++sym_index) + { + /// We are not interested in empty symbols. + if (!elf_sym[sym_index].st_size) + continue; + + /* Get the name of the sym_index-th symbol. + * This is located at the address of st_name relative to the beginning of the string table. + */ + const char * sym_name = &strtab[elf_sym[sym_index].st_name]; + + if (!sym_name) + continue; + + SymbolIndex::Symbol symbol; + symbol.address_begin = reinterpret_cast(info->dlpi_addr + elf_sym[sym_index].st_value); + symbol.address_end = reinterpret_cast(info->dlpi_addr + elf_sym[sym_index].st_value + elf_sym[sym_index].st_size); + symbol.name = sym_name; + symbols.push_back(std::move(symbol)); + } + + break; + } + } + } +} + + +void collectSymbolsFromELFSymbolTable( + dl_phdr_info * info, + const Elf & elf, + const Elf::Section & symbol_table, + const Elf::Section & string_table, + std::vector & symbols) +{ + /// Iterate symbol table. + const ElfSym * symbol_table_entry = reinterpret_cast(symbol_table.begin()); + const ElfSym * symbol_table_end = reinterpret_cast(symbol_table.end()); + + const char * strings = string_table.begin(); + + for (; symbol_table_entry < symbol_table_end; ++symbol_table_entry) + { + if (!symbol_table_entry->st_name + || !symbol_table_entry->st_value + || !symbol_table_entry->st_size + || strings + symbol_table_entry->st_name >= elf.end()) + continue; + + /// Find the name in strings table. + const char * symbol_name = strings + symbol_table_entry->st_name; + + if (!symbol_name) + continue; + + SymbolIndex::Symbol symbol; + symbol.address_begin = reinterpret_cast(info->dlpi_addr + symbol_table_entry->st_value); + symbol.address_end = reinterpret_cast(info->dlpi_addr + symbol_table_entry->st_value + symbol_table_entry->st_size); + symbol.name = symbol_name; + symbols.push_back(std::move(symbol)); + } +} + + +bool searchAndCollectSymbolsFromELFSymbolTable( + dl_phdr_info * info, + const Elf & elf, + unsigned section_header_type, + const char * string_table_name, + std::vector & symbols) +{ + std::optional symbol_table; + std::optional string_table; + + if (!elf.iterateSections([&](const Elf::Section & section, size_t) + { + if (section.header.sh_type == section_header_type) + symbol_table.emplace(section); + else if (section.header.sh_type == SHT_STRTAB && 0 == strcmp(section.name(), string_table_name)) + string_table.emplace(section); + + if (symbol_table && string_table) + return true; + return false; + })) + { + return false; + } + + collectSymbolsFromELFSymbolTable(info, elf, *symbol_table, *string_table, symbols); + return true; +} + + +void collectSymbolsFromELF(dl_phdr_info * info, + std::vector & symbols, + std::vector & objects) +{ + std::string object_name = info->dlpi_name; + + /// If the name is empty - it's main executable. + /// Find a elf file for the main executable. + + if (object_name.empty()) + object_name = "/proc/self/exe"; + + std::error_code ec; + std::filesystem::path canonical_path = std::filesystem::canonical(object_name, ec); + + if (ec) + return; + + /// Debug info and symbol table sections may be splitted to separate binary. + std::filesystem::path debug_info_path = std::filesystem::path("/usr/lib/debug") / canonical_path; + + object_name = std::filesystem::exists(debug_info_path) ? debug_info_path : canonical_path; + + SymbolIndex::Object object; + object.elf = std::make_unique(object_name); + object.address_begin = reinterpret_cast(info->dlpi_addr); + object.address_end = reinterpret_cast(info->dlpi_addr + object.elf->size()); + object.name = object_name; + objects.push_back(std::move(object)); + + searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_SYMTAB, ".strtab", symbols); + + /// Unneeded because they were parsed from "program headers" of loaded objects. + //searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_DYNSYM, ".dynstr", symbols); +} + + +/* Callback for dl_iterate_phdr. + * Is called by dl_iterate_phdr for every loaded shared lib until something + * else than 0 is returned by one call of this function. + */ +int collectSymbols(dl_phdr_info * info, size_t, void * data_ptr) +{ + SymbolIndex::Data & data = *reinterpret_cast(data_ptr); + + collectSymbolsFromProgramHeaders(info, data.symbols); + collectSymbolsFromELF(info, data.symbols, data.objects); + + /* Continue iterations */ + return 0; +} + + +template +const T * find(const void * address, const std::vector & vec) +{ + /// First range that has left boundary greater than address. + + auto it = std::lower_bound(vec.begin(), vec.end(), address, + [](const T & symbol, const void * addr) { return symbol.address_begin <= addr; }); + + if (it == vec.begin()) + return nullptr; + else + --it; /// Last range that has left boundary less or equals than address. + + if (address >= it->address_begin && address < it->address_end) + return &*it; + else + return nullptr; +} + +} + + +void SymbolIndex::update() +{ + dl_iterate_phdr(collectSymbols, &data.symbols); + + std::sort(data.objects.begin(), data.objects.end(), [](const Object & a, const Object & b) { return a.address_begin < b.address_begin; }); + std::sort(data.symbols.begin(), data.symbols.end(), [](const Symbol & a, const Symbol & b) { return a.address_begin < b.address_begin; }); + + /// We found symbols both from loaded program headers and from ELF symbol tables. + data.symbols.erase(std::unique(data.symbols.begin(), data.symbols.end(), [](const Symbol & a, const Symbol & b) + { + return a.address_begin == b.address_begin && a.address_end == b.address_end; + }), data.symbols.end()); +} + +const SymbolIndex::Symbol * SymbolIndex::findSymbol(const void * address) const +{ + return find(address, data.symbols); +} + +const SymbolIndex::Object * SymbolIndex::findObject(const void * address) const +{ + return find(address, data.objects); +} + +} diff --git a/dbms/src/Common/SymbolIndex.h b/dbms/src/Common/SymbolIndex.h new file mode 100644 index 00000000000..41a773f5f4a --- /dev/null +++ b/dbms/src/Common/SymbolIndex.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/** Allow to quickly find symbol name from address. + * Used as a replacement for "dladdr" function which is extremely slow. + * It works better than "dladdr" because it also allows to search private symbols, that are not participated in shared linking. + */ +class SymbolIndex : public ext::singleton +{ +protected: + friend class ext::singleton; + SymbolIndex() { update(); } + +public: + struct Symbol + { + const void * address_begin; + const void * address_end; + const char * name; + }; + + struct Object + { + const void * address_begin; + const void * address_end; + std::string name; + std::unique_ptr elf; + }; + + const Symbol * findSymbol(const void * address) const; + const Object * findObject(const void * address) const; + + const std::vector & symbols() const { return data.symbols; } + const std::vector & objects() const { return data.objects; } + + struct Data + { + std::vector symbols; + std::vector objects; + }; +private: + Data data; + + void update(); +}; + +} diff --git a/dbms/src/Common/TraceCollector.cpp b/dbms/src/Common/TraceCollector.cpp index bfb49c4ef75..5a508619eb3 100644 --- a/dbms/src/Common/TraceCollector.cpp +++ b/dbms/src/Common/TraceCollector.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index 7a7ec42f6ff..caebc59ce7f 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -4,14 +4,13 @@ #include "TestKeeper.h" #include -#include #include #include #include #include #include -#include +#include #include #include @@ -159,8 +158,7 @@ struct ZooKeeperArgs } /// Shuffle the hosts to distribute the load among ZooKeeper nodes. - pcg64 rng(randomSeed()); - std::shuffle(hosts_strings.begin(), hosts_strings.end(), rng); + std::shuffle(hosts_strings.begin(), hosts_strings.end(), thread_local_rng); for (auto & host : hosts_strings) { diff --git a/dbms/src/Common/tests/CMakeLists.txt b/dbms/src/Common/tests/CMakeLists.txt index 83d69b2c8f2..d11a46a38d9 100644 --- a/dbms/src/Common/tests/CMakeLists.txt +++ b/dbms/src/Common/tests/CMakeLists.txt @@ -78,3 +78,6 @@ target_link_libraries (stopwatch PRIVATE clickhouse_common_io) add_executable (mi_malloc_test mi_malloc_test.cpp) target_link_libraries (mi_malloc_test PRIVATE clickhouse_common_io) + +add_executable (symbol_index symbol_index.cpp) +target_link_libraries (symbol_index PRIVATE clickhouse_common_io) diff --git a/dbms/src/Common/tests/symbol_index.cpp b/dbms/src/Common/tests/symbol_index.cpp new file mode 100644 index 00000000000..6c0d303fe35 --- /dev/null +++ b/dbms/src/Common/tests/symbol_index.cpp @@ -0,0 +1,58 @@ +#include +#include +#include +#include +#include +#include +#include + + +NO_INLINE const void * getAddress() +{ + return __builtin_return_address(0); +} + +using namespace DB; + +int main(int argc, char ** argv) +{ + if (argc < 2) + { + std::cerr << "Usage: ./symbol_index address\n"; + return 1; + } + + const SymbolIndex & symbol_index = SymbolIndex::instance(); + + for (const auto & elem : symbol_index.symbols()) + std::cout << elem.name << ": " << elem.address_begin << " ... " << elem.address_end << "\n"; + std::cout << "\n"; + + const void * address = reinterpret_cast(std::stoull(argv[1], nullptr, 16)); + + auto symbol = symbol_index.findSymbol(address); + if (symbol) + std::cerr << symbol->name << ": " << symbol->address_begin << " ... " << symbol->address_end << "\n"; + else + std::cerr << "SymbolIndex: Not found\n"; + + Dl_info info; + if (dladdr(address, &info) && info.dli_sname) + std::cerr << demangle(info.dli_sname) << ": " << info.dli_saddr << "\n"; + else + std::cerr << "dladdr: Not found\n"; + + auto object = symbol_index.findObject(getAddress()); + Dwarf dwarf(*object->elf); + + Dwarf::LocationInfo location; + if (dwarf.findAddress(uintptr_t(address), location, Dwarf::LocationInfoMode::FAST)) + std::cerr << location.file.toString() << ":" << location.line << "\n"; + else + std::cerr << "Dwarf: Not found\n"; + + std::cerr << "\n"; + std::cerr << StackTrace().toString() << "\n"; + + return 0; +} diff --git a/dbms/src/Common/thread_local_rng.cpp b/dbms/src/Common/thread_local_rng.cpp new file mode 100644 index 00000000000..031b2123481 --- /dev/null +++ b/dbms/src/Common/thread_local_rng.cpp @@ -0,0 +1,4 @@ +#include +#include + +thread_local pcg64 thread_local_rng{randomSeed()}; diff --git a/dbms/src/Common/thread_local_rng.h b/dbms/src/Common/thread_local_rng.h new file mode 100644 index 00000000000..990f6f64948 --- /dev/null +++ b/dbms/src/Common/thread_local_rng.h @@ -0,0 +1,5 @@ +#pragma once +#include + +/// Fairly good thread-safe random number generator, but probably slow-down thread creation a little. +extern thread_local pcg64 thread_local_rng; diff --git a/dbms/src/Compression/LZ4_decompress_faster.h b/dbms/src/Compression/LZ4_decompress_faster.h index 68a55c04581..ff29c205276 100644 --- a/dbms/src/Compression/LZ4_decompress_faster.h +++ b/dbms/src/Compression/LZ4_decompress_faster.h @@ -101,6 +101,7 @@ struct PerformanceStatistics Element data[NUM_ELEMENTS]; + /// It's Ok that generator is not seeded. pcg64 rng; /// To select from different algorithms we use a kind of "bandits" algorithm. diff --git a/dbms/src/Core/MySQLProtocol.h b/dbms/src/Core/MySQLProtocol.h index cb0e6ab001c..ccd127352ed 100644 --- a/dbms/src/Core/MySQLProtocol.h +++ b/dbms/src/Core/MySQLProtocol.h @@ -246,26 +246,17 @@ class PacketPayloadWriteBuffer : public WriteBuffer { public: PacketPayloadWriteBuffer(WriteBuffer & out, size_t payload_length, uint8_t & sequence_id) - : WriteBuffer(out.position(), 0) - , out(out) - , sequence_id(sequence_id) - , total_left(payload_length) + : WriteBuffer(out.position(), 0), out(out), sequence_id(sequence_id), total_left(payload_length) { - startPacket(); + startNewPacket(); + setWorkingBuffer(); + pos = out.position(); } - void checkPayloadSize() + bool remainingPayloadSize() { - if (bytes_written + offset() < payload_length) - { - std::stringstream ss; - ss << "Incomplete payload. Written " << bytes << " bytes, expected " << payload_length << " bytes."; - throw Exception(ss.str(), 0); - - } + return total_left; } - - ~PacketPayloadWriteBuffer() override { next(); } private: WriteBuffer & out; uint8_t & sequence_id; @@ -273,8 +264,9 @@ private: size_t total_left = 0; size_t payload_length = 0; size_t bytes_written = 0; + bool eof = false; - void startPacket() + void startNewPacket() { payload_length = std::min(total_left, MAX_PACKET_LENGTH); bytes_written = 0; @@ -282,33 +274,38 @@ private: out.write(reinterpret_cast(&payload_length), 3); out.write(sequence_id++); - - working_buffer = WriteBuffer::Buffer(out.position(), out.position() + std::min(payload_length - bytes_written, out.available())); - pos = working_buffer.begin(); + bytes += 4; } + + /// Sets working buffer to the rest of current packet payload. + void setWorkingBuffer() + { + out.nextIfAtEnd(); + working_buffer = WriteBuffer::Buffer(out.position(), out.position() + std::min(payload_length - bytes_written, out.available())); + + if (payload_length - bytes_written == 0) + { + /// Finished writing packet. Due to an implementation of WriteBuffer, working_buffer cannot be empty. Further write attempts will throw Exception. + eof = true; + working_buffer.resize(1); + } + } + protected: void nextImpl() override { - int written = pos - working_buffer.begin(); + const int written = pos - working_buffer.begin(); + if (eof) + throw Exception("Cannot write after end of buffer.", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); + out.position() += written; bytes_written += written; - if (bytes_written < payload_length) - { - out.nextIfAtEnd(); - working_buffer = WriteBuffer::Buffer(out.position(), out.position() + std::min(payload_length - bytes_written, out.available())); - } - else if (total_left > 0 || payload_length == MAX_PACKET_LENGTH) - { - // Starting new packet, since packets of size greater than MAX_PACKET_LENGTH should be split. - startPacket(); - } - else - { - // Finished writing packet. Buffer is set to empty to prevent rewriting (pos will be set to the beginning of a working buffer in next()). - // Further attempts to write will stall in the infinite loop. - working_buffer = WriteBuffer::Buffer(out.position(), out.position()); - } + /// Packets of size greater than MAX_PACKET_LENGTH are split into few packets of size MAX_PACKET_LENGTH and las packet of size < MAX_PACKET_LENGTH. + if (bytes_written == payload_length && (total_left > 0 || payload_length == MAX_PACKET_LENGTH)) + startNewPacket(); + + setWorkingBuffer(); } }; @@ -320,7 +317,13 @@ public: { PacketPayloadWriteBuffer buf(buffer, getPayloadSize(), sequence_id); writePayloadImpl(buf); - buf.checkPayloadSize(); + buf.next(); + if (buf.remainingPayloadSize()) + { + std::stringstream ss; + ss << "Incomplete payload. Written " << getPayloadSize() - buf.remainingPayloadSize() << " bytes, expected " << getPayloadSize() << " bytes."; + throw Exception(ss.str(), 0); + } } virtual ~WritePacket() = default; diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 60f2599c73f..0af0bf02d2e 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -334,6 +334,7 @@ struct Settings : public SettingsCollection \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \ + M(SettingBool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.") \ \ M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.") \ M(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value") \ diff --git a/dbms/src/DataStreams/CubeBlockInputStream.cpp b/dbms/src/DataStreams/CubeBlockInputStream.cpp index c32378d97e6..50a6c0a970b 100644 --- a/dbms/src/DataStreams/CubeBlockInputStream.cpp +++ b/dbms/src/DataStreams/CubeBlockInputStream.cpp @@ -36,43 +36,58 @@ Block CubeBlockInputStream::getHeader() const Block CubeBlockInputStream::readImpl() { - /** After reading a block from input stream, + /** After reading all blocks from input stream, * we will calculate all subsets of columns on next iterations of readImpl * by zeroing columns at positions, where bits are zero in current bitmask. */ - if (mask) + + if (!is_data_read) { - --mask; - Block cube_block = source_block; - for (size_t i = 0; i < keys.size(); ++i) + BlocksList source_blocks; + while (auto block = children[0]->read()) + source_blocks.push_back(block); + + if (source_blocks.empty()) + return {}; + + is_data_read = true; + mask = (1 << keys.size()) - 1; + + if (source_blocks.size() > 1) + source_block = aggregator.mergeBlocks(source_blocks, false); + else + source_block = std::move(source_blocks.front()); + + zero_block = source_block.cloneEmpty(); + for (auto key : keys) { - if (!((mask >> i) & 1)) - { - size_t pos = keys.size() - i - 1; - auto & current = cube_block.getByPosition(keys[pos]); - current.column = zero_block.getByPosition(keys[pos]).column; - } + auto & current = zero_block.getByPosition(key); + current.column = current.column->cloneResized(source_block.rows()); } - BlocksList cube_blocks = { cube_block }; - Block finalized = aggregator.mergeBlocks(cube_blocks, true); + auto finalized = source_block; + finalizeBlock(finalized); return finalized; } - source_block = children[0]->read(); - if (!source_block) - return source_block; + if (!mask) + return {}; - zero_block = source_block.cloneEmpty(); - for (auto key : keys) + --mask; + auto cube_block = source_block; + + for (size_t i = 0; i < keys.size(); ++i) { - auto & current = zero_block.getByPosition(key); - current.column = current.column->cloneResized(source_block.rows()); + if (!((mask >> i) & 1)) + { + size_t pos = keys.size() - i - 1; + auto & current = cube_block.getByPosition(keys[pos]); + current.column = zero_block.getByPosition(keys[pos]).column; + } } - Block finalized = source_block; - finalizeBlock(finalized); - mask = (1 << keys.size()) - 1; + BlocksList cube_blocks = { cube_block }; + Block finalized = aggregator.mergeBlocks(cube_blocks, true); return finalized; } } diff --git a/dbms/src/DataStreams/CubeBlockInputStream.h b/dbms/src/DataStreams/CubeBlockInputStream.h index 2f435a6031c..7e62950e8ee 100644 --- a/dbms/src/DataStreams/CubeBlockInputStream.h +++ b/dbms/src/DataStreams/CubeBlockInputStream.h @@ -36,6 +36,7 @@ private: UInt32 mask = 0; Block source_block; Block zero_block; + bool is_data_read = false; }; } diff --git a/dbms/src/DataStreams/IBlockInputStream.cpp b/dbms/src/DataStreams/IBlockInputStream.cpp index 406a660879c..a2c3fb2247c 100644 --- a/dbms/src/DataStreams/IBlockInputStream.cpp +++ b/dbms/src/DataStreams/IBlockInputStream.cpp @@ -255,6 +255,10 @@ static void limitProgressingSpeed(size_t total_progress_size, size_t max_speed_i if (desired_microseconds > total_elapsed_microseconds) { UInt64 sleep_microseconds = desired_microseconds - total_elapsed_microseconds; + + /// Never sleep more than one second (it should be enough to limit speed for a reasonable amount, and otherwise it's too easy to make query hang). + sleep_microseconds = std::min(UInt64(1000000), sleep_microseconds); + sleepForMicroseconds(sleep_microseconds); ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_microseconds); @@ -349,7 +353,7 @@ void IBlockInputStream::progressImpl(const Progress & value) ErrorCodes::TOO_SLOW); /// If the predicted execution time is longer than `max_execution_time`. - if (limits.max_execution_time != 0 && total_rows) + if (limits.max_execution_time != 0 && total_rows && progress.read_rows) { double estimated_execution_time_seconds = elapsed_seconds * (static_cast(total_rows) / progress.read_rows); diff --git a/dbms/src/DataStreams/RollupBlockInputStream.cpp b/dbms/src/DataStreams/RollupBlockInputStream.cpp index e43aa51e617..a913dc727fa 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.cpp +++ b/dbms/src/DataStreams/RollupBlockInputStream.cpp @@ -33,26 +33,40 @@ Block RollupBlockInputStream::readImpl() * by zeroing out every column one-by-one and re-merging a block. */ - if (current_key >= 0) + if (!is_data_read) { - auto & current = rollup_block.getByPosition(keys[current_key]); - current.column = current.column->cloneEmpty()->cloneResized(rollup_block.rows()); - --current_key; + BlocksList source_blocks; + while (auto block = children[0]->read()) + source_blocks.push_back(block); - BlocksList rollup_blocks = { rollup_block }; - rollup_block = aggregator.mergeBlocks(rollup_blocks, false); + if (source_blocks.empty()) + return {}; - Block finalized = rollup_block; + is_data_read = true; + if (source_blocks.size() > 1) + rollup_block = aggregator.mergeBlocks(source_blocks, false); + else + rollup_block = std::move(source_blocks.front()); + + current_key = keys.size() - 1; + + auto finalized = rollup_block; finalizeBlock(finalized); return finalized; } - Block block = children[0]->read(); - current_key = keys.size() - 1; + if (current_key < 0) + return {}; - rollup_block = block; - finalizeBlock(block); + auto & current = rollup_block.getByPosition(keys[current_key]); + current.column = current.column->cloneEmpty()->cloneResized(rollup_block.rows()); + --current_key; - return block; + BlocksList rollup_blocks = { rollup_block }; + rollup_block = aggregator.mergeBlocks(rollup_blocks, false); + + auto finalized = rollup_block; + finalizeBlock(finalized); + return finalized; } } diff --git a/dbms/src/DataStreams/RollupBlockInputStream.h b/dbms/src/DataStreams/RollupBlockInputStream.h index 1c1e29e7a13..dabf1e392a3 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.h +++ b/dbms/src/DataStreams/RollupBlockInputStream.h @@ -35,6 +35,7 @@ private: ColumnNumbers keys; ssize_t current_key = -1; Block rollup_block; + bool is_data_read = false; }; } diff --git a/dbms/src/DataStreams/narrowBlockInputStreams.cpp b/dbms/src/DataStreams/narrowBlockInputStreams.cpp index 07c26bc9b67..ede12019d25 100644 --- a/dbms/src/DataStreams/narrowBlockInputStreams.cpp +++ b/dbms/src/DataStreams/narrowBlockInputStreams.cpp @@ -1,6 +1,5 @@ #include -#include -#include +#include #include @@ -21,8 +20,7 @@ BlockInputStreams narrowBlockInputStreams(BlockInputStreams & inputs, size_t wid for (size_t i = 0; i < size; ++i) distribution[i] = i % width; - pcg64 generator(randomSeed()); - std::shuffle(distribution.begin(), distribution.end(), generator); + std::shuffle(distribution.begin(), distribution.end(), thread_local_rng); for (size_t i = 0; i < size; ++i) partitions[distribution[i]].push_back(inputs[i]); diff --git a/dbms/src/Formats/MySQLWireBlockOutputStream.cpp b/dbms/src/Formats/MySQLWireBlockOutputStream.cpp index d7766ffe61a..f032d5b84d4 100644 --- a/dbms/src/Formats/MySQLWireBlockOutputStream.cpp +++ b/dbms/src/Formats/MySQLWireBlockOutputStream.cpp @@ -55,16 +55,18 @@ void MySQLWireBlockOutputStream::write(const Block & block) void MySQLWireBlockOutputStream::writeSuffix() { - QueryStatus * process_list_elem = context.getProcessListElement(); - CurrentThread::finalizePerformanceCounters(); - QueryStatusInfo info = process_list_elem->getInfo(); - size_t affected_rows = info.written_rows; - + size_t affected_rows = 0; std::stringstream human_readable_info; - human_readable_info << std::fixed << std::setprecision(3) - << "Read " << info.read_rows << " rows, " << formatReadableSizeWithBinarySuffix(info.read_bytes) << " in " << info.elapsed_seconds << " sec., " - << static_cast(info.read_rows / info.elapsed_seconds) << " rows/sec., " - << formatReadableSizeWithBinarySuffix(info.read_bytes / info.elapsed_seconds) << "/sec."; + if (QueryStatus * process_list_elem = context.getProcessListElement()) + { + CurrentThread::finalizePerformanceCounters(); + QueryStatusInfo info = process_list_elem->getInfo(); + affected_rows = info.written_rows; + human_readable_info << std::fixed << std::setprecision(3) + << "Read " << info.read_rows << " rows, " << formatReadableSizeWithBinarySuffix(info.read_bytes) << " in " << info.elapsed_seconds << " sec., " + << static_cast(info.read_rows / info.elapsed_seconds) << " rows/sec., " + << formatReadableSizeWithBinarySuffix(info.read_bytes / info.elapsed_seconds) << "/sec."; + } if (header.columns() == 0) packet_sender.sendPacket(OK_Packet(0x0, context.mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true); diff --git a/dbms/src/Functions/addressToLine.cpp b/dbms/src/Functions/addressToLine.cpp new file mode 100644 index 00000000000..7f7bd609dee --- /dev/null +++ b/dbms/src/Functions/addressToLine.cpp @@ -0,0 +1,151 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int FUNCTION_NOT_ALLOWED; +} + +class FunctionAddressToLine : public IFunction +{ +public: + static constexpr auto name = "addressToLine"; + static FunctionPtr create(const Context & context) + { + if (!context.getSettingsRef().allow_introspection_functions) + throw Exception("Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); + + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1) + throw Exception("Function " + getName() + " needs exactly one argument; passed " + + toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto & type = arguments[0].type; + + if (!WhichDataType(type.get()).isUInt64()) + throw Exception("The only argument for function " + getName() + " must be UInt64. Found " + + type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const ColumnPtr & column = block.getByPosition(arguments[0]).column; + const ColumnUInt64 * column_concrete = checkAndGetColumn(column.get()); + + if (!column_concrete) + throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + const typename ColumnVector::Container & data = column_concrete->getData(); + auto result_column = ColumnString::create(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + StringRef res_str = implCached(data[i]); + result_column->insertData(res_str.data, res_str.size); + } + + block.getByPosition(result).column = std::move(result_column); + } + +private: + std::mutex mutex; + Arena arena; + using Map = HashMap; + Map map; + std::unordered_map dwarfs; + + StringRef impl(uintptr_t addr) + { + const SymbolIndex & symbol_index = SymbolIndex::instance(); + + if (auto object = symbol_index.findObject(reinterpret_cast(addr))) + { + auto dwarf_it = dwarfs.try_emplace(object->name, *object->elf).first; + if (!std::filesystem::exists(object->name)) + return {}; + + Dwarf::LocationInfo location; + if (dwarf_it->second.findAddress(addr - uintptr_t(object->address_begin), location, Dwarf::LocationInfoMode::FAST)) + { + const char * arena_begin = nullptr; + WriteBufferFromArena out(arena, arena_begin); + + writeString(location.file.toString(), out); + writeChar(':', out); + writeIntText(location.line, out); + + StringRef out_str = out.finish(); + out_str.data = arena.insert(out_str.data, out_str.size); + return out_str; + } + else + { + return object->name; + } + } + else + return {}; + } + + StringRef implCached(uintptr_t addr) + { + Map::iterator it; + bool inserted; + std::lock_guard lock(mutex); + map.emplace(addr, it, inserted); + if (inserted) + it->getSecond() = impl(addr); + return it->getSecond(); + } +}; + +void registerFunctionAddressToLine(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/symbolizeAddress.cpp b/dbms/src/Functions/addressToSymbol.cpp similarity index 61% rename from dbms/src/Functions/symbolizeAddress.cpp rename to dbms/src/Functions/addressToSymbol.cpp index 1096a8924b3..ceb641e457c 100644 --- a/dbms/src/Functions/symbolizeAddress.cpp +++ b/dbms/src/Functions/addressToSymbol.cpp @@ -1,15 +1,11 @@ -#include -#include -#include -#include -#include -#include +#include #include #include #include #include #include #include +#include #include @@ -20,17 +16,19 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int FUNCTION_NOT_ALLOWED; } -class FunctionSymbolizeAddress : public IFunction +class FunctionAddressToSymbol : public IFunction { public: - static constexpr auto name = "symbolizeAddress"; - static FunctionPtr create(const Context &) + static constexpr auto name = "addressToSymbol"; + static FunctionPtr create(const Context & context) { - return std::make_shared(); + if (!context.getSettingsRef().allow_introspection_functions) + throw Exception("Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); + return std::make_shared(); } String getName() const override @@ -63,25 +61,10 @@ public: return true; } - static std::string addressToSymbol(UInt64 uint_address) - { - void * addr = unalignedLoad(&uint_address); - - /// This is extremely slow. - Dl_info info; - if (dladdr(addr, &info) && info.dli_sname) - { - int demangling_status = 0; - return demangle(info.dli_sname, demangling_status); - } - else - { - return {}; - } - } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { + const SymbolIndex & symbol_index = SymbolIndex::instance(); + const ColumnPtr & column = block.getByPosition(arguments[0]).column; const ColumnUInt64 * column_concrete = checkAndGetColumn(column.get()); @@ -91,25 +74,21 @@ public: const typename ColumnVector::Container & data = column_concrete->getData(); auto result_column = ColumnString::create(); - static SimpleCache func_cached; - for (size_t i = 0; i < input_rows_count; ++i) { - std::string symbol = func_cached(data[i]); - result_column->insertDataWithTerminatingZero(symbol.data(), symbol.size() + 1); + if (const auto * symbol = symbol_index.findSymbol(reinterpret_cast(data[i]))) + result_column->insertDataWithTerminatingZero(symbol->name, strlen(symbol->name) + 1); + else + result_column->insertDefault(); } block.getByPosition(result).column = std::move(result_column); - - /// Do not let our cache to grow indefinitely (simply drop it) - if (func_cached.size() > 1000000) - func_cached.drop(); } }; -void registerFunctionSymbolizeAddress(FunctionFactory & factory) +void registerFunctionAddressToSymbol(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Functions/demange.cpp b/dbms/src/Functions/demange.cpp new file mode 100644 index 00000000000..a94b99f62ea --- /dev/null +++ b/dbms/src/Functions/demange.cpp @@ -0,0 +1,93 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int FUNCTION_NOT_ALLOWED; +} + +class FunctionDemangle : public IFunction +{ +public: + static constexpr auto name = "demangle"; + static FunctionPtr create(const Context & context) + { + if (!context.getSettingsRef().allow_introspection_functions) + throw Exception("Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); + + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1) + throw Exception("Function " + getName() + " needs exactly one argument; passed " + + toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto & type = arguments[0].type; + + if (!WhichDataType(type.get()).isString()) + throw Exception("The only argument for function " + getName() + " must be String. Found " + + type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const ColumnPtr & column = block.getByPosition(arguments[0]).column; + const ColumnString * column_concrete = checkAndGetColumn(column.get()); + + if (!column_concrete) + throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + auto result_column = ColumnString::create(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + StringRef source = column_concrete->getDataAt(i); + int status = 0; + std::string demangled = demangle(source.data, status); + result_column->insertDataWithTerminatingZero(demangled.data(), demangled.size() + 1); + } + + block.getByPosition(result).column = std::move(result_column); + } +}; + +void registerFunctionDemangle(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index 178f085e1ad..eba9a96e5e0 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -40,7 +40,7 @@ void registerFunctionsIntrospection(FunctionFactory &); void registerFunctionsNull(FunctionFactory &); void registerFunctionsFindCluster(FunctionFactory &); void registerFunctionsJSON(FunctionFactory &); -void registerFunctionSymbolizeAddress(FunctionFactory &); +void registerFunctionsIntrospection(FunctionFactory &); void registerFunctions() { @@ -79,7 +79,7 @@ void registerFunctions() registerFunctionsNull(factory); registerFunctionsFindCluster(factory); registerFunctionsJSON(factory); - registerFunctionSymbolizeAddress(factory); + registerFunctionsIntrospection(factory); } } diff --git a/dbms/src/Functions/registerFunctionsIntrospection.cpp b/dbms/src/Functions/registerFunctionsIntrospection.cpp new file mode 100644 index 00000000000..448400b37ab --- /dev/null +++ b/dbms/src/Functions/registerFunctionsIntrospection.cpp @@ -0,0 +1,18 @@ +namespace DB +{ + +class FunctionFactory; + +void registerFunctionAddressToSymbol(FunctionFactory & factory); +void registerFunctionDemangle(FunctionFactory & factory); +void registerFunctionAddressToLine(FunctionFactory & factory); + +void registerFunctionsIntrospection(FunctionFactory & factory) +{ + registerFunctionAddressToSymbol(factory); + registerFunctionDemangle(factory); + registerFunctionAddressToLine(factory); +} + +} + diff --git a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp new file mode 100644 index 00000000000..68e04b45d99 --- /dev/null +++ b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -0,0 +1,210 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_JOIN_ON_EXPRESSION; + extern const int AMBIGUOUS_COLUMN_NAME; + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; +} + +void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, + const std::pair & table_no) +{ + ASTPtr left = left_ast->clone(); + ASTPtr right = right_ast->clone(); + + if (table_no.first == 1 || table_no.second == 2) + analyzed_join.addOnKeys(left, right); + else if (table_no.first == 2 || table_no.second == 1) + analyzed_join.addOnKeys(right, left); + else + throw Exception("Cannot detect left and right JOIN keys. JOIN ON section is ambiguous.", + ErrorCodes::AMBIGUOUS_COLUMN_NAME); + has_some = true; +} + +void CollectJoinOnKeysMatcher::Data::addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, + const std::pair & table_no) +{ + if (table_no.first == 1 || table_no.second == 2) + { + asof_left_key = left_ast->clone(); + asof_right_key = right_ast->clone(); + return; + } + + throw Exception("ASOF JOIN for (left_table.x <= right_table.x) is not implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +void CollectJoinOnKeysMatcher::Data::asofToJoinKeys() +{ + if (!asof_left_key || !asof_right_key) + throw Exception("No inequality in ASOF JOIN ON section.", ErrorCodes::INVALID_JOIN_ON_EXPRESSION); + addJoinKeys(asof_left_key, asof_right_key, {1, 2}); +} + + +void CollectJoinOnKeysMatcher::visit(const ASTFunction & func, const ASTPtr & ast, Data & data) +{ + if (func.name == "and") + return; /// go into children + + if (func.name == "equals") + { + ASTPtr left = func.arguments->children.at(0); + ASTPtr right = func.arguments->children.at(1); + auto table_numbers = getTableNumbers(ast, left, right, data); + data.addJoinKeys(left, right, table_numbers); + return; + } + + bool less_or_equals = (func.name == "lessOrEquals"); + bool greater_or_equals = (func.name == "greaterOrEquals"); + + if (data.is_asof && (less_or_equals || greater_or_equals)) + { + if (data.asof_left_key || data.asof_right_key) + throwSyntaxException("ASOF JOIN expects exactly one inequality in ON section, unexpected " + queryToString(ast) + "."); + + ASTPtr left = func.arguments->children.at(0); + ASTPtr right = func.arguments->children.at(1); + auto table_numbers = getTableNumbers(ast, left, right, data); + + if (greater_or_equals) + data.addAsofJoinKeys(left, right, table_numbers); + else + data.addAsofJoinKeys(right, left, std::make_pair(table_numbers.second, table_numbers.first)); + + return; + } + + throwSyntaxException("Expected equals expression, got " + queryToString(ast) + "."); +} + +void CollectJoinOnKeysMatcher::getIdentifiers(const ASTPtr & ast, std::vector & out) +{ + if (const auto * ident = ast->as()) + { + if (IdentifierSemantic::getColumnName(*ident)) + out.push_back(ident); + return; + } + + for (const auto & child : ast->children) + getIdentifiers(child, out); +} + +std::pair CollectJoinOnKeysMatcher::getTableNumbers(const ASTPtr & expr, const ASTPtr & left_ast, const ASTPtr & right_ast, + Data & data) +{ + std::vector left_identifiers; + std::vector right_identifiers; + + getIdentifiers(left_ast, left_identifiers); + getIdentifiers(right_ast, right_identifiers); + + size_t left_idents_table = getTableForIdentifiers(left_identifiers, data); + size_t right_idents_table = getTableForIdentifiers(right_identifiers, data); + + if (left_idents_table && left_idents_table == right_idents_table) + { + auto left_name = queryToString(*left_identifiers[0]); + auto right_name = queryToString(*right_identifiers[0]); + + throwSyntaxException("In expression " + queryToString(expr) + " columns " + left_name + " and " + right_name + + " are from the same table but from different arguments of equal function."); + } + + return std::make_pair(left_idents_table, right_idents_table); +} + +const ASTIdentifier * CollectJoinOnKeysMatcher::unrollAliases(const ASTIdentifier * identifier, const Aliases & aliases) +{ + if (identifier->compound()) + return identifier; + + UInt32 max_attempts = 100; + for (auto it = aliases.find(identifier->name); it != aliases.end();) + { + const ASTIdentifier * parent = identifier; + identifier = it->second->as(); + if (!identifier) + break; /// not a column alias + if (identifier == parent) + break; /// alias to itself with the same name: 'a as a' + if (identifier->compound()) + break; /// not an alias. Break to prevent cycle through short names: 'a as b, t1.b as a' + + it = aliases.find(identifier->name); + if (!max_attempts--) + throw Exception("Cannot unroll aliases for '" + identifier->name + "'", ErrorCodes::LOGICAL_ERROR); + } + + return identifier; +} + +/// @returns 1 if identifiers belongs to left table, 2 for right table and 0 if unknown. Throws on table mix. +/// Place detected identifier into identifiers[0] if any. +size_t CollectJoinOnKeysMatcher::getTableForIdentifiers(std::vector & identifiers, const Data & data) +{ + size_t table_number = 0; + + for (auto & ident : identifiers) + { + const ASTIdentifier * identifier = unrollAliases(ident, data.aliases); + if (!identifier) + continue; + + /// Column name could be cropped to a short form in TranslateQualifiedNamesVisitor. + /// In this case it saves membership in IdentifierSemantic. + size_t membership = IdentifierSemantic::getMembership(*identifier); + + if (!membership) + { + const String & name = identifier->name; + bool in_left_table = data.source_columns.count(name); + bool in_right_table = data.joined_columns.count(name); + + if (in_left_table && in_right_table) + throw Exception("Column '" + name + "' is ambiguous", ErrorCodes::AMBIGUOUS_COLUMN_NAME); + + if (in_left_table) + membership = 1; + if (in_right_table) + membership = 2; + } + + if (membership && table_number == 0) + { + table_number = membership; + std::swap(ident, identifiers[0]); /// move first detected identifier to the first position + } + + if (membership && membership != table_number) + { + throw Exception("Invalid columns in JOIN ON section. Columns " + + identifiers[0]->getAliasOrColumnName() + " and " + ident->getAliasOrColumnName() + + " are from different tables.", ErrorCodes::INVALID_JOIN_ON_EXPRESSION); + } + } + + return table_number; +} + +[[noreturn]] void CollectJoinOnKeysMatcher::throwSyntaxException(const String & msg) +{ + throw Exception("Invalid expression for JOIN ON. " + msg + + " Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) " + "[AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]", + ErrorCodes::INVALID_JOIN_ON_EXPRESSION); +} + +} diff --git a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h index 7dc3051167a..bae6781a18a 100644 --- a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -1,23 +1,16 @@ #pragma once +#include #include -#include - #include #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int INVALID_JOIN_ON_EXPRESSION; - extern const int AMBIGUOUS_COLUMN_NAME; - extern const int LOGICAL_ERROR; -} - +class ASTIdentifier; +struct AnalyzedJoin; class CollectJoinOnKeysMatcher { @@ -30,7 +23,14 @@ public: const NameSet & source_columns; const NameSet & joined_columns; const Aliases & aliases; - bool has_some = false; + const bool is_asof; + ASTPtr asof_left_key{}; + ASTPtr asof_right_key{}; + bool has_some{false}; + + void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, const std::pair & table_no); + void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, const std::pair & table_no); + void asofToJoinKeys(); }; static void visit(const ASTPtr & ast, Data & data) @@ -48,146 +48,14 @@ public: } private: - static void visit(const ASTFunction & func, const ASTPtr & ast, Data & data) - { - if (func.name == "and") - return; /// go into children + static void visit(const ASTFunction & func, const ASTPtr & ast, Data & data); - if (func.name == "equals") - { - ASTPtr left = func.arguments->children.at(0)->clone(); - ASTPtr right = func.arguments->children.at(1)->clone(); - addJoinKeys(ast, left, right, data); - return; - } + static void getIdentifiers(const ASTPtr & ast, std::vector & out); + static std::pair getTableNumbers(const ASTPtr & expr, const ASTPtr & left_ast, const ASTPtr & right_ast, Data & data); + static const ASTIdentifier * unrollAliases(const ASTIdentifier * identifier, const Aliases & aliases); + static size_t getTableForIdentifiers(std::vector & identifiers, const Data & data); - throwSyntaxException("Expected equals expression, got " + queryToString(ast) + "."); - } - - static void getIdentifiers(const ASTPtr & ast, std::vector & out) - { - if (const auto * ident = ast->as()) - { - if (IdentifierSemantic::getColumnName(*ident)) - out.push_back(ident); - return; - } - - for (const auto & child : ast->children) - getIdentifiers(child, out); - } - - static void addJoinKeys(const ASTPtr & expr, ASTPtr left_ast, ASTPtr right_ast, Data & data) - { - std::vector left_identifiers; - std::vector right_identifiers; - - getIdentifiers(left_ast, left_identifiers); - getIdentifiers(right_ast, right_identifiers); - - size_t left_idents_table = getTableForIdentifiers(left_identifiers, data); - size_t right_idents_table = getTableForIdentifiers(right_identifiers, data); - - if (left_idents_table && left_idents_table == right_idents_table) - { - auto left_name = queryToString(*left_identifiers[0]); - auto right_name = queryToString(*right_identifiers[0]); - - throwSyntaxException("In expression " + queryToString(expr) + " columns " + left_name + " and " + right_name - + " are from the same table but from different arguments of equal function."); - } - - if (left_idents_table == 1 || right_idents_table == 2) - data.analyzed_join.addOnKeys(left_ast, right_ast); - else if (left_idents_table == 2 || right_idents_table == 1) - data.analyzed_join.addOnKeys(right_ast, left_ast); - else - throw Exception("Cannot detect left and right JOIN keys. JOIN ON section is ambiguous.", - ErrorCodes::AMBIGUOUS_COLUMN_NAME); - - data.has_some = true; - } - - static const ASTIdentifier * unrollAliases(const ASTIdentifier * identifier, const Aliases & aliases) - { - if (identifier->compound()) - return identifier; - - UInt32 max_attempts = 100; - for (auto it = aliases.find(identifier->name); it != aliases.end();) - { - const ASTIdentifier * parent = identifier; - identifier = it->second->as(); - if (!identifier) - break; /// not a column alias - if (identifier == parent) - break; /// alias to itself with the same name: 'a as a' - if (identifier->compound()) - break; /// not an alias. Break to prevent cycle through short names: 'a as b, t1.b as a' - - it = aliases.find(identifier->name); - if (!max_attempts--) - throw Exception("Cannot unroll aliases for '" + identifier->name + "'", ErrorCodes::LOGICAL_ERROR); - } - - return identifier; - } - - /// @returns 1 if identifiers belongs to left table, 2 for right table and 0 if unknown. Throws on table mix. - /// Place detected identifier into identifiers[0] if any. - static size_t getTableForIdentifiers(std::vector & identifiers, const Data & data) - { - size_t table_number = 0; - - for (auto & ident : identifiers) - { - const ASTIdentifier * identifier = unrollAliases(ident, data.aliases); - if (!identifier) - continue; - - /// Column name could be cropped to a short form in TranslateQualifiedNamesVisitor. - /// In this case it saves membership in IdentifierSemantic. - size_t membership = IdentifierSemantic::getMembership(*identifier); - - if (!membership) - { - const String & name = identifier->name; - bool in_left_table = data.source_columns.count(name); - bool in_right_table = data.joined_columns.count(name); - - if (in_left_table && in_right_table) - throw Exception("Column '" + name + "' is ambiguous", ErrorCodes::AMBIGUOUS_COLUMN_NAME); - - if (in_left_table) - membership = 1; - if (in_right_table) - membership = 2; - } - - if (membership && table_number == 0) - { - table_number = membership; - std::swap(ident, identifiers[0]); /// move first detected identifier to the first position - } - - if (membership && membership != table_number) - { - throw Exception("Invalid columns in JOIN ON section. Columns " - + identifiers[0]->getAliasOrColumnName() + " and " + ident->getAliasOrColumnName() - + " are from different tables.", ErrorCodes::INVALID_JOIN_ON_EXPRESSION); - } - } - - return table_number; - } - - [[noreturn]] static void throwSyntaxException(const String & msg) - { - throw Exception("Invalid expression for JOIN ON. " + msg + - " Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) " - "[AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]", - ErrorCodes::INVALID_JOIN_ON_EXPRESSION); - } + [[noreturn]] static void throwSyntaxException(const String & msg); }; /// Parse JOIN ON expression and collect ASTs for joined columns. diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index cec36f42469..91196bd53a1 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -5,12 +5,12 @@ #include #include #include -#include #include #include #include #include #include +#include #include #include #include @@ -50,7 +50,7 @@ #include #include #include -#include +#include #include #include #include @@ -205,8 +205,6 @@ struct ContextShared Context::ApplicationType application_type = Context::ApplicationType::SERVER; - pcg64 rng{randomSeed()}; - /// vector of xdbc-bridge commands, they will be killed when Context will be destroyed std::vector> bridge_commands; @@ -1172,12 +1170,8 @@ void Context::setCurrentQueryId(const String & query_id) } words; } random; - { - auto lock = getLock(); - - random.words.a = shared->rng(); - random.words.b = shared->rng(); - } + random.words.a = thread_local_rng(); + random.words.b = thread_local_rng(); /// Use protected constructor. struct UUID : Poco::UUID diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index def39d4d91c..5a13477147c 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -406,6 +406,15 @@ CancellationCode ProcessList::sendCancelToQuery(const String & current_query_id, } +void ProcessList::killAllQueries() +{ + std::lock_guard lock(mutex); + + for (auto & process : processes) + process.cancelQuery(true); +} + + QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const { QueryStatusInfo res; diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index b75a4e7a730..4cdf7c18fea 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -315,6 +315,8 @@ public: /// Try call cancel() for input and output streams of query with specified id and user CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill = false); + + void killAllQueries(); }; } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 04102f5ae15..02156b20995 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include @@ -509,11 +510,14 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & s for (const auto & col : analyzed_join.columns_from_joined_table) joined_columns.insert(col.original_name); - CollectJoinOnKeysVisitor::Data data{analyzed_join, source_columns, joined_columns, aliases}; + bool is_asof = (table_join.strictness == ASTTableJoin::Strictness::Asof); + CollectJoinOnKeysVisitor::Data data{analyzed_join, source_columns, joined_columns, aliases, is_asof}; CollectJoinOnKeysVisitor(data).visit(table_join.on_expression); if (!data.has_some) throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression), ErrorCodes::INVALID_JOIN_ON_EXPRESSION); + if (is_asof) + data.asofToJoinKeys(); } bool make_nullable = join_use_nulls && isLeftOrFull(table_join.kind); diff --git a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 3ddaed53687..446fd687503 100644 --- a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -71,16 +71,18 @@ void MySQLOutputFormat::consume(Chunk chunk) void MySQLOutputFormat::finalize() { - QueryStatus * process_list_elem = context.getProcessListElement(); - CurrentThread::finalizePerformanceCounters(); - QueryStatusInfo info = process_list_elem->getInfo(); - size_t affected_rows = info.written_rows; - + size_t affected_rows = 0; std::stringstream human_readable_info; - human_readable_info << std::fixed << std::setprecision(3) - << "Read " << info.read_rows << " rows, " << formatReadableSizeWithBinarySuffix(info.read_bytes) << " in " << info.elapsed_seconds << " sec., " - << static_cast(info.read_rows / info.elapsed_seconds) << " rows/sec., " - << formatReadableSizeWithBinarySuffix(info.read_bytes / info.elapsed_seconds) << "/sec."; + if (QueryStatus * process_list_elem = context.getProcessListElement()) + { + CurrentThread::finalizePerformanceCounters(); + QueryStatusInfo info = process_list_elem->getInfo(); + affected_rows = info.written_rows; + human_readable_info << std::fixed << std::setprecision(3) + << "Read " << info.read_rows << " rows, " << formatReadableSizeWithBinarySuffix(info.read_bytes) << " in " << info.elapsed_seconds << " sec., " + << static_cast(info.read_rows / info.elapsed_seconds) << " rows/sec., " + << formatReadableSizeWithBinarySuffix(info.read_bytes / info.elapsed_seconds) << "/sec."; + } auto & header = getPort(PortKind::Main).getHeader(); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 934d651fead..64141a509b5 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -162,14 +163,6 @@ static const auto MUTATIONS_FINALIZING_SLEEP_MS = 1 * 1000; extern const int MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER = 5 * 60; -/** For randomized selection of replicas. */ -/// avoid error: non-local variable 'DB::rng' declared '__thread' needs dynamic initialization -#ifndef __APPLE__ -thread_local -#endif - pcg64 rng{randomSeed()}; - - void StorageReplicatedMergeTree::setZooKeeper(zkutil::ZooKeeperPtr zookeeper) { std::lock_guard lock(current_zookeeper_mutex); @@ -708,7 +701,7 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: part->columns, part->checksums); Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); - std::shuffle(replicas.begin(), replicas.end(), rng); + std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); bool has_been_already_added = false; for (const String & replica : replicas) @@ -2445,7 +2438,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); /// Select replicas in uniformly random order. - std::shuffle(replicas.begin(), replicas.end(), rng); + std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); for (const String & replica : replicas) { @@ -2470,7 +2463,7 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entr Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); /// Select replicas in uniformly random order. - std::shuffle(replicas.begin(), replicas.end(), rng); + std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); for (const String & replica : replicas) { @@ -2529,7 +2522,7 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart( Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); /// Select replicas in uniformly random order. - std::shuffle(replicas.begin(), replicas.end(), rng); + std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); String largest_part_found; String largest_replica_found; diff --git a/dbms/src/Storages/System/StorageSystemContributors.cpp b/dbms/src/Storages/System/StorageSystemContributors.cpp index 6e165a4ee40..f1a3c68efc5 100644 --- a/dbms/src/Storages/System/StorageSystemContributors.cpp +++ b/dbms/src/Storages/System/StorageSystemContributors.cpp @@ -2,8 +2,7 @@ #include #include -#include -#include +#include extern const char * auto_contributors[]; @@ -23,8 +22,7 @@ void StorageSystemContributors::fillData(MutableColumns & res_columns, const Con for (auto it = auto_contributors; *it; ++it) contributors.emplace_back(*it); - pcg64 rng(randomSeed()); - std::shuffle(contributors.begin(), contributors.end(), rng); + std::shuffle(contributors.begin(), contributors.end(), thread_local_rng); for (auto & it : contributors) res_columns[0]->insert(String(it)); diff --git a/dbms/tests/performance/number_formatting_formats.xml b/dbms/tests/performance/number_formatting_formats.xml index 144c8aaba00..df83c5cbf11 100644 --- a/dbms/tests/performance/number_formatting_formats.xml +++ b/dbms/tests/performance/number_formatting_formats.xml @@ -35,6 +35,7 @@ Parquet ODBCDriver2 Null + MySQLWire diff --git a/dbms/tests/performance/select_format.xml b/dbms/tests/performance/select_format.xml index 0182401c640..c5ad1acd396 100644 --- a/dbms/tests/performance/select_format.xml +++ b/dbms/tests/performance/select_format.xml @@ -44,6 +44,7 @@ Native XML ODBCDriver2 + MySQLWire diff --git a/dbms/tests/queries/0_stateless/00701_rollup.reference b/dbms/tests/queries/0_stateless/00701_rollup.reference index ec07ad52cae..637ae0bcb52 100644 --- a/dbms/tests/queries/0_stateless/00701_rollup.reference +++ b/dbms/tests/queries/0_stateless/00701_rollup.reference @@ -25,3 +25,13 @@ a 70 4 b 50 4 120 8 + 120 8 +a 70 4 +b 50 4 + 0 120 8 +a 0 70 4 +a 1 25 2 +a 2 45 2 +b 0 50 4 +b 1 15 2 +b 2 35 2 diff --git a/dbms/tests/queries/0_stateless/00701_rollup.sql b/dbms/tests/queries/0_stateless/00701_rollup.sql index 3f4df923f90..fa7f3a21657 100644 --- a/dbms/tests/queries/0_stateless/00701_rollup.sql +++ b/dbms/tests/queries/0_stateless/00701_rollup.sql @@ -1,14 +1,9 @@ DROP TABLE IF EXISTS rollup; CREATE TABLE rollup(a String, b Int32, s Int32) ENGINE = Memory; -INSERT INTO rollup VALUES('a', 1, 10); -INSERT INTO rollup VALUES('a', 1, 15); -INSERT INTO rollup VALUES('a', 2, 20); -INSERT INTO rollup VALUES('a', 2, 25); -INSERT INTO rollup VALUES('b', 1, 10); -INSERT INTO rollup VALUES('b', 1, 5); -INSERT INTO rollup VALUES('b', 2, 20); -INSERT INTO rollup VALUES('b', 2, 15); +INSERT INTO rollup VALUES ('a', 1, 10), ('a', 1, 15), ('a', 2, 20); +INSERT INTO rollup VALUES ('a', 2, 25), ('b', 1, 10), ('b', 1, 5); +INSERT INTO rollup VALUES ('b', 2, 20), ('b', 2, 15); SELECT a, b, sum(s), count() from rollup GROUP BY ROLLUP(a, b) ORDER BY a, b; @@ -20,4 +15,9 @@ SELECT a, sum(s), count() from rollup GROUP BY a WITH ROLLUP ORDER BY a; SELECT a, sum(s), count() from rollup GROUP BY a WITH ROLLUP WITH TOTALS ORDER BY a; +SET group_by_two_level_threshold = 1; + +SELECT a, sum(s), count() from rollup GROUP BY a WITH ROLLUP ORDER BY a; +SELECT a, b, sum(s), count() from rollup GROUP BY a, b WITH ROLLUP ORDER BY a, b; + DROP TABLE rollup; diff --git a/dbms/tests/queries/0_stateless/00720_with_cube.reference b/dbms/tests/queries/0_stateless/00720_with_cube.reference index a0b951978f9..818e8626dde 100644 --- a/dbms/tests/queries/0_stateless/00720_with_cube.reference +++ b/dbms/tests/queries/0_stateless/00720_with_cube.reference @@ -18,8 +18,8 @@ b 1 15 2 b 2 35 2 0 120 8 - 1 40 4 0 120 8 + 1 40 4 2 80 4 a 0 70 4 a 1 25 2 @@ -27,8 +27,8 @@ a 2 45 2 b 0 50 4 b 1 15 2 b 2 35 2 - 1 40 4 0 120 8 + 1 40 4 2 80 4 a 0 70 4 a 1 25 2 @@ -38,3 +38,12 @@ b 1 15 2 b 2 35 2 0 120 8 + 0 120 8 + 1 40 4 + 2 80 4 +a 0 70 4 +a 1 25 2 +a 2 45 2 +b 0 50 4 +b 1 15 2 +b 2 35 2 diff --git a/dbms/tests/queries/0_stateless/00720_with_cube.sql b/dbms/tests/queries/0_stateless/00720_with_cube.sql index bcde617803e..42b65c8222c 100644 --- a/dbms/tests/queries/0_stateless/00720_with_cube.sql +++ b/dbms/tests/queries/0_stateless/00720_with_cube.sql @@ -1,21 +1,21 @@ -DROP TABLE IF EXISTS rollup; -CREATE TABLE rollup(a String, b Int32, s Int32) ENGINE = Memory; +DROP TABLE IF EXISTS cube; +CREATE TABLE cube(a String, b Int32, s Int32) ENGINE = Memory; -INSERT INTO rollup VALUES('a', 1, 10); -INSERT INTO rollup VALUES('a', 1, 15); -INSERT INTO rollup VALUES('a', 2, 20); -INSERT INTO rollup VALUES('a', 2, 25); -INSERT INTO rollup VALUES('b', 1, 10); -INSERT INTO rollup VALUES('b', 1, 5); -INSERT INTO rollup VALUES('b', 2, 20); -INSERT INTO rollup VALUES('b', 2, 15); +-- SET experimental_use_processors=1; -SELECT a, b, sum(s), count() from rollup GROUP BY CUBE(a, b) ORDER BY a, b; +INSERT INTO cube VALUES ('a', 1, 10), ('a', 1, 15), ('a', 2, 20); +INSERT INTO cube VALUES ('a', 2, 25), ('b', 1, 10), ('b', 1, 5); +INSERT INTO cube VALUES ('b', 2, 20), ('b', 2, 15); -SELECT a, b, sum(s), count() from rollup GROUP BY CUBE(a, b) WITH TOTALS ORDER BY a, b; +SELECT a, b, sum(s), count() from cube GROUP BY CUBE(a, b) ORDER BY a, b; -SELECT a, b, sum(s), count() from rollup GROUP BY a, b WITH CUBE ORDER BY a; +SELECT a, b, sum(s), count() from cube GROUP BY CUBE(a, b) WITH TOTALS ORDER BY a, b; -SELECT a, b, sum(s), count() from rollup GROUP BY a, b WITH CUBE WITH TOTALS ORDER BY a; +SELECT a, b, sum(s), count() from cube GROUP BY a, b WITH CUBE ORDER BY a, b; -DROP TABLE rollup; +SELECT a, b, sum(s), count() from cube GROUP BY a, b WITH CUBE WITH TOTALS ORDER BY a, b; + +SET group_by_two_level_threshold = 1; +SELECT a, b, sum(s), count() from cube GROUP BY a, b WITH CUBE ORDER BY a, b; + +DROP TABLE cube; diff --git a/dbms/tests/queries/0_stateless/00927_asof_join_noninclusive.sql b/dbms/tests/queries/0_stateless/00927_asof_join_noninclusive.sql index 50644352b64..5f15f3b593d 100644 --- a/dbms/tests/queries/0_stateless/00927_asof_join_noninclusive.sql +++ b/dbms/tests/queries/0_stateless/00927_asof_join_noninclusive.sql @@ -12,7 +12,7 @@ INSERT INTO B(k,t,b) VALUES (2,3,3); SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t); -SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF INNER JOIN B ON A.k == B.k AND A.t == B.t ORDER BY (A.k, A.t); +SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF INNER JOIN B ON A.k == B.k AND A.t >= B.t ORDER BY (A.k, A.t); SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF JOIN B USING(k,t) ORDER BY (A.k, A.t); diff --git a/dbms/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.reference b/dbms/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.reference new file mode 100644 index 00000000000..33891b98730 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.reference @@ -0,0 +1,2 @@ + "rows_read": 16384, + "rows_read": 16384, diff --git a/dbms/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh b/dbms/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh new file mode 100755 index 00000000000..cea48890320 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS lowString;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS string;" + +$CLICKHOUSE_CLIENT -n --query=" +create table lowString +( +a LowCardinality(String), +b Date +) +ENGINE = MergeTree() +PARTITION BY toYYYYMM(b) +ORDER BY (a)" + +$CLICKHOUSE_CLIENT -n --query=" +create table string +( +a String, +b Date +) +ENGINE = MergeTree() +PARTITION BY toYYYYMM(b) +ORDER BY (a)" + +$CLICKHOUSE_CLIENT --query="insert into lowString (a, b) select top 100000 toString(number), today() from system.numbers" + +$CLICKHOUSE_CLIENT --query="insert into string (a, b) select top 100000 toString(number), today() from system.numbers" + +$CLICKHOUSE_CLIENT --query="select count() from lowString where a in ('1', '2') FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="select count() from string where a in ('1', '2') FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="DROP TABLE lowString;" +$CLICKHOUSE_CLIENT --query="DROP TABLE string;" diff --git a/dbms/tests/queries/0_stateless/00974_query_profiler.sql b/dbms/tests/queries/0_stateless/00974_query_profiler.sql index b3d70bc6ac3..d77e6564782 100644 --- a/dbms/tests/queries/0_stateless/00974_query_profiler.sql +++ b/dbms/tests/queries/0_stateless/00974_query_profiler.sql @@ -1,9 +1,11 @@ +SET allow_introspection_functions = 1; + SET query_profiler_real_time_period_ns = 100000000; SET log_queries = 1; SELECT sleep(0.5), ignore('test real time query profiler'); SET log_queries = 0; SYSTEM FLUSH LOGS; -WITH symbolizeAddress(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test real time query profiler%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%FunctionSleep%'; +WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test real time query profiler%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%FunctionSleep%'; SET query_profiler_real_time_period_ns = 0; SET query_profiler_cpu_time_period_ns = 100000000; @@ -11,4 +13,4 @@ SET log_queries = 1; SELECT count(), ignore('test cpu time query profiler') FROM numbers(1000000000); SET log_queries = 0; SYSTEM FLUSH LOGS; -WITH symbolizeAddress(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test cpu time query profiler%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%Numbers%'; +WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test cpu time query profiler%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%Numbers%'; diff --git a/dbms/tests/queries/0_stateless/00976_asof_join_on.reference b/dbms/tests/queries/0_stateless/00976_asof_join_on.reference new file mode 100644 index 00000000000..ffa8117cc75 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00976_asof_join_on.reference @@ -0,0 +1,13 @@ +1 1 0 0 +1 2 1 2 +1 3 1 2 +2 1 0 0 +2 2 0 0 +2 3 2 3 +3 1 0 0 +3 2 0 0 +3 3 0 0 +9 +1 2 1 2 +1 3 1 2 +2 3 2 3 diff --git a/dbms/tests/queries/0_stateless/00976_asof_join_on.sql b/dbms/tests/queries/0_stateless/00976_asof_join_on.sql new file mode 100644 index 00000000000..740287b7c30 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00976_asof_join_on.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS A; +DROP TABLE IF EXISTS B; + +CREATE TABLE A(a UInt32, t UInt32) ENGINE = Memory; +CREATE TABLE B(b UInt32, t UInt32) ENGINE = Memory; + +INSERT INTO A (a,t) VALUES (1,1),(1,2),(1,3), (2,1),(2,2),(2,3), (3,1),(3,2),(3,3); +INSERT INTO B (b,t) VALUES (1,2),(1,4),(2,3); + +SELECT A.a, A.t, B.b, B.t FROM A ASOF LEFT JOIN B ON A.a == B.b AND A.t >= B.t ORDER BY (A.a, A.t); +SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t; +SELECT A.a, A.t, B.b, B.t FROM A ASOF INNER JOIN B ON B.t <= A.t AND A.a == B.b; +SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t <= B.t; -- { serverError 48 } +SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND B.t >= A.t; -- { serverError 48 } +SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t > B.t; -- { serverError 403 } +SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t < B.t; -- { serverError 403 } +SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t == B.t; -- { serverError 403 } +SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t != B.t; -- { serverError 403 } + +DROP TABLE A; +DROP TABLE B; diff --git a/dbms/tests/queries/0_stateless/00976_max_execution_speed.reference b/dbms/tests/queries/0_stateless/00976_max_execution_speed.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00976_max_execution_speed.sql b/dbms/tests/queries/0_stateless/00976_max_execution_speed.sql new file mode 100644 index 00000000000..06386d77413 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00976_max_execution_speed.sql @@ -0,0 +1,2 @@ +SET max_execution_speed = 1, max_execution_time = 3; +SELECT count() FROM system.numbers; -- { serverError 159 } diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 578d7c1bb64..01b71df817a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -813,7 +813,7 @@ Default value: 1. ## count_distinct_implementation {#settings-count_distinct_implementation} -Specifies which of the `uniq*` functions should be used for performing the [COUNT(DISTINCT ...)](../../query_language/agg_functions/reference.md#agg_function-count) construction. +Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT ...)](../../query_language/agg_functions/reference.md#agg_function-count) construction. Possible values: diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index e63a9115270..6c99c299aa5 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -8,12 +8,12 @@ They are located in the 'system' database. ## system.asynchronous_metrics {#system_tables-asynchronous_metrics} -Contains metrics which are calculated periodically in background. For example, the amount of RAM in use. +Contains metrics that are calculated periodically in the background. For example, the amount of RAM in use. Columns: -- `metric` ([String](../data_types/string.md)) — Metric's name. -- `value` ([Float64](../data_types/float.md)) — Metric's value. +- `metric` ([String](../data_types/string.md)) — Metric name. +- `value` ([Float64](../data_types/float.md)) — Metric value. **Example** @@ -40,7 +40,7 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 - [Monitoring](monitoring.md) — Base concepts of ClickHouse monitoring. - [system.metrics](#system_tables-metrics) — Contains instantly calculated metrics. -- [system.events](#system_tables-events) — Contains a number of happened events. +- [system.events](#system_tables-events) — Contains a number of events that have occurred. ## system.clusters @@ -48,7 +48,7 @@ Contains information about clusters available in the config file and the servers Columns: ``` -cluster String — The cluster name. +cluster String — The cluster name. shard_num UInt32 — The shard number in the cluster, starting from 1. shard_weight UInt32 — The relative weight of the shard when writing data. replica_num UInt32 — The replica number in the shard, starting from 1. @@ -119,13 +119,13 @@ Note that the amount of memory used by the dictionary is not proportional to the ## system.events {#system_tables-events} -Contains information about the number of events that have occurred in the system. For example, in the table, you can find how many `SELECT` queries are processed from the moment of ClickHouse server start. +Contains information about the number of events that have occurred in the system. For example, in the table, you can find how many `SELECT` queries were processed since the ClickHouse server started. Columns: - `event` ([String](../data_types/string.md)) — Event name. -- `value` ([UInt64](../data_types/int_uint.md)) — Count of events occurred. -- `description` ([String](../data_types/string.md)) — Description of an event. +- `value` ([UInt64](../data_types/int_uint.md)) — Number of events occurred. +- `description` ([String](../data_types/string.md)) — Event description. **Example** @@ -135,11 +135,11 @@ SELECT * FROM system.events LIMIT 5 ```text ┌─event─────────────────────────────────┬─value─┬─description────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ Query │ 12 │ Number of queries started to be interpreted and maybe executed. Does not include queries that are failed to parse, that are rejected due to AST size limits; rejected due to quota limits or limits on number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. │ +│ Query │ 12 │ Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. │ │ SelectQuery │ 8 │ Same as Query, but only for SELECT queries. │ │ FileOpen │ 73 │ Number of files opened. │ │ ReadBufferFromFileDescriptorRead │ 155 │ Number of reads (read/pread) from a file descriptor. Does not include sockets. │ -│ ReadBufferFromFileDescriptorReadBytes │ 9931 │ Number of bytes read from file descriptors. If the file is compressed, this will show compressed data size. │ +│ ReadBufferFromFileDescriptorReadBytes │ 9931 │ Number of bytes read from file descriptors. If the file is compressed, this will show the compressed data size. │ └───────────────────────────────────────┴───────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` @@ -197,13 +197,13 @@ Columns: ## system.metrics {#system_tables-metrics} -Contains metrics which can be calculated instantly, or have an current value. For example, a number of simultaneously processed queries, the current value for replica delay. This table is always up to date. +Contains metrics which can be calculated instantly, or have a current value. For example, the number of simultaneously processed queries or the current replica delay. This table is always up to date. Columns: -- `metric` ([String](../data_types/string.md)) — Metric's name. -- `value` ([Int64](../data_types/int_uint.md)) — Metric's value. -- `description` ([String](../data_types/string.md)) — Description of the metric. +- `metric` ([String](../data_types/string.md)) — Metric name. +- `value` ([Int64](../data_types/int_uint.md)) — Metric value. +- `description` ([String](../data_types/string.md)) — Metric description. **Example** @@ -216,19 +216,19 @@ SELECT * FROM system.metrics LIMIT 10 │ Query │ 1 │ Number of executing queries │ │ Merge │ 0 │ Number of executing background merges │ │ PartMutation │ 0 │ Number of mutations (ALTER DELETE/UPDATE) │ -│ ReplicatedFetch │ 0 │ Number of data parts fetching from replica │ -│ ReplicatedSend │ 0 │ Number of data parts sending to replicas │ +│ ReplicatedFetch │ 0 │ Number of data parts being fetched from replicas │ +│ ReplicatedSend │ 0 │ Number of data parts being sent to replicas │ │ ReplicatedChecks │ 0 │ Number of data parts checking for consistency │ -│ BackgroundPoolTask │ 0 │ Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches or replication queue bookkeeping) │ -│ BackgroundSchedulePoolTask │ 0 │ Number of active tasks in BackgroundSchedulePool. This pool is used for periodic tasks of ReplicatedMergeTree like cleaning old data parts, altering data parts, replica re-initialization, etc. │ -│ DiskSpaceReservedForMerge │ 0 │ Disk space reserved for currently running background merges. It is slightly more than total size of currently merging parts. │ -│ DistributedSend │ 0 │ Number of connections sending data, that was INSERTed to Distributed tables, to remote servers. Both synchronous and asynchronous mode. │ +│ BackgroundPoolTask │ 0 │ Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches, or replication queue bookkeeping) │ +│ BackgroundSchedulePoolTask │ 0 │ Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc. │ +│ DiskSpaceReservedForMerge │ 0 │ Disk space reserved for currently running background merges. It is slightly more than the total size of currently merging parts. │ +│ DistributedSend │ 0 │ Number of connections to remote servers sending data that was INSERTed into Distributed tables. Both synchronous and asynchronous mode. │ └────────────────────────────┴───────┴──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` **See Also** - [system.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [system.events](#system_tables-events) — Contains a umber of happened events. +- [system.events](#system_tables-events) — Contains a number of events that occurred. - [Monitoring](monitoring.md) — Base concepts of ClickHouse monitoring. ## system.numbers @@ -368,7 +368,7 @@ To enable query logging, set the parameter [log_queries](settings/settings.md#se The `system.query_log` table registers two kinds of queries: 1. Initial queries, that were run directly by the client. -2. Child queries that were initiated by other queries (for distributed query execution). For such a kind of queries, information about the parent queries is shown in the `initial_*` columns. +2. Child queries that were initiated by other queries (for distributed query execution). For such queries, information about parent queries is shown in the `initial_*` columns. Columns: @@ -391,9 +391,7 @@ Columns: - `query` (String) — Query string. - `exception` (String) — Exception message. - `stack_trace` (String) — Stack trace (a list of methods called before the error occurred). An empty string, if the query is completed successfully. -- `is_initial_query` (UInt8) — Kind of query. Possible values: - - 1 — Query was initiated by the client. - - 0 — Query was initiated by another query for distributed query execution. +- `is_initial_query` (UInt8) — Flag that indicates whether the query is initiated by the client (1), or by another query for distributed query execution (0). - `user` (String) — Name of the user initiated the current query. - `query_id` (String) — ID of the query. - `address` (FixedString(16)) — IP address the query was initiated from. diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index 76fb90d4fc4..ce9f2b6cdbb 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -6,7 +6,7 @@ Counts the number of rows or not-NULL values. ClickHouse supports the following syntaxes for `count`: - `count(expr)` or `COUNT(DISTINCT expr)`. -- `count()` or `COUNT(*)`. The `count()` syntax is a ClickHouse-specific implementation. +- `count()` or `COUNT(*)`. The `count()` syntax is ClickHouse-specific. **Parameters** @@ -19,15 +19,15 @@ The function can take: **Returned value** - If the function is called without parameters it counts the number of rows. -- If the [expression](../syntax.md#syntax-expressions) is passed, then the function counts how many times this expression returned not null. If the expression returns a value of the [Nullable](../../data_types/nullable.md) data type, then the result of `count` stays not `Nullable`. The function returns 0 if the expression returned `NULL` for all the rows. +- If the [expression](../syntax.md#syntax-expressions) is passed, then the function counts how many times this expression returned not null. If the expression returns a [Nullable](../../data_types/nullable.md)-type value, then the result of `count` stays not `Nullable`. The function returns 0 if the expression returned `NULL` for all the rows. In both cases the type of the returned value is [UInt64](../../data_types/int_uint.md). **Details** -ClickHouse supports the `COUNT(DISTINCT ...)` syntax. The behavior of this construction depends on the [count_distinct_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) setting. It defines which of the [uniq*](#agg_function-uniq) functions is used to perform the operation. By default the [uniqExact](#agg_function-uniqexact) function. +ClickHouse supports the `COUNT(DISTINCT ...)` syntax. The behavior of this construction depends on the [count_distinct_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) setting. It defines which of the [uniq*](#agg_function-uniq) functions is used to perform the operation. The default is the [uniqExact](#agg_function-uniqexact) function. -A `SELECT count() FROM table` query is not optimized, because the number of entries in the table is not stored separately. It chooses some small column from the table and count the number of values in it. +The `SELECT count() FROM table` query is not optimized, because the number of entries in the table is not stored separately. It chooses a small column from the table and counts the number of values in it. **Examples** @@ -62,7 +62,7 @@ SELECT count(DISTINCT num) FROM t └────────────────┘ ``` -This example shows that `count(DISTINCT num)` is performed by the function `uniqExact` corresponding to the `count_distinct_implementation` setting value. +This example shows that `count(DISTINCT num)` is performed by the `uniqExact` function according to the `count_distinct_implementation` setting value. ## any(x) {#agg_function-any} @@ -522,24 +522,24 @@ uniq(x[, ...]) **Parameters** -Function takes the variable number of parameters. Parameters can be of types: `Tuple`, `Array`, `Date`, `DateTime`, `String`, numeric types. +The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types. **Returned value** -- The number of the [UInt64](../../data_types/int_uint.md) type. +- A [UInt64](../../data_types/int_uint.md)-type number. **Implementation details** Function: - Calculates a hash for all parameters in the aggregate, then uses it in calculations. -- Uses an adaptive sampling algorithm. For the calculation state, the function uses a sample of element hash values with a size up to 65536. +- Uses an adaptive sampling algorithm. For the calculation state, the function uses a sample of element hash values up to 65536. - This algorithm is very accurate and very efficient on CPU. When query contains several of these functions, using `uniq` is almost as fast as using other aggregate functions. + This algorithm is very accurate and very efficient on the CPU. When the query contains several of these functions, using `uniq` is almost as fast as using other aggregate functions. -- Provides the result deterministically (it doesn't depend on the order of query processing). +- Provides the result deterministically (it doesn't depend on the query processing order). -We recommend to use this function in almost all scenarios. +We recommend using this function in almost all scenarios. **See Also** @@ -549,40 +549,40 @@ We recommend to use this function in almost all scenarios. ## uniqCombined {#agg_function-uniqcombined} -Calculates the approximate number of different values of the argument. +Calculates the approximate number of different argument values. ``` uniqCombined(HLL_precision)(x[, ...]) ``` -The `uniqCombined` function is a good choice for calculating the number of different values, but keep in mind that the estimation error for large sets (200 million elements and more) will become larger than theoretical value due to poor choice of hash function. +The `uniqCombined` function is a good choice for calculating the number of different values, but keep in mind that the estimation error for large sets (200 million elements and more) will be larger than the theoretical value due to the poor hash function choice. **Parameters** -Function takes the variable number of parameters. Parameters can be of types: `Tuple`, `Array`, `Date`, `DateTime`, `String`, numeric types. +The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types. -`HLL_precision` is the base-2 logarithm of the number of cells in [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Optional, you can use the function as `uniqCombined(x[, ...])`. The default value for `HLL_precision` is 17, that is effectively 96 KiB of space (2^17 cells of 6 bits each). +`HLL_precision` is the base-2 logarithm of the number of cells in [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Optional, you can use the function as `uniqCombined(x[, ...])`. The default value for `HLL_precision` is 17, which is effectively 96 KiB of space (2^17 cells, 6 bits each). **Returned value** -- The number of the [UInt64](../../data_types/int_uint.md) type. +- A number [UInt64](../../data_types/int_uint.md)-type number. **Implementation details** Function: - Calculates a hash for all parameters in the aggregate, then uses it in calculations. -- Uses combination of three algorithms: array, hash table and HyperLogLog with an error correction table. +- Uses a combination of three algorithms: array, hash table, and HyperLogLog with an error correction table. - For small number of distinct elements, the array is used. When the set size becomes larger the hash table is used, while it is smaller than HyperLogLog data structure. For larger number of elements, the HyperLogLog is used, and it will occupy fixed amount of memory. + For a small number of distinct elements, an array is used. When the set size is larger, a hash table is used. For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory. -- Provides the result deterministically (it doesn't depend on the order of query processing). +- Provides the result deterministically (it doesn't depend on the query processing order). -In comparison with the [uniq](#agg_function-uniq) function the `uniqCombined`: +Compared to the [uniq](#agg_function-uniq) function, the `uniqCombined`: - Consumes several times less memory. - Calculates with several times higher accuracy. -- Performs slightly lower usually. In some scenarios `uniqCombined` can perform better than `uniq`, for example, with distributed queries that transmit a large number of aggregation states over the network. +- Usually has slightly lower performance. In some scenarios, `uniqCombined` can perform better than `uniq`, for example, with distributed queries that transmit a large number of aggregation states over the network. **See Also** @@ -593,7 +593,7 @@ In comparison with the [uniq](#agg_function-uniq) function the `uniqCombined`: ## uniqHLL12 {#agg_function-uniqhll12} -Calculates the approximate number of different values of the argument, using the [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) algortithm. +Calculates the approximate number of different argument values, using the [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) algorithm. ``` uniqHLL12(x[, ...]) @@ -601,22 +601,22 @@ uniqHLL12(x[, ...]) **Parameters** -Function takes the variable number of parameters. Parameters can be of types: `Tuple`, `Array`, `Date`, `DateTime`, `String`, numeric types. +The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types. **Returned value** -- The number of the [UInt64](../../data_types/int_uint.md) type. +- A [UInt64](../../data_types/int_uint.md)-type number. **Implementation details** Function: - Calculates a hash for all parameters in the aggregate, then uses it in calculations. -- Uses the HyperLogLog algorithm to approximate the number of different values of the argument. +- Uses the HyperLogLog algorithm to approximate the number of different argument values. 212 5-bit cells are used. The size of the state is slightly more than 2.5 KB. The result is not very accurate (up to ~10% error) for small data sets (<10K elements). However, the result is fairly accurate for high-cardinality data sets (10K-100M), with a maximum error of ~1.6%. Starting from 100M, the estimation error increases, and the function will return very inaccurate results for data sets with extremely high cardinality (1B+ elements). -- Provides the determinate result (it doesn't depend on the order of query processing). +- Provides the determinate result (it doesn't depend on the query processing order). We don't recommend using this function. In most cases, use the [uniq](#agg_function-uniq) or [uniqCombined](#agg_function-uniqcombined) function. @@ -629,19 +629,19 @@ We don't recommend using this function. In most cases, use the [uniq](#agg_funct ## uniqExact {#agg_function-uniqexact} -Calculates the exact number of different values of the argument. +Calculates the exact number of different argument values. ``` uniqExact(x[, ...]) ``` -Use the `uniqExact` function if you definitely need an exact result. Otherwise use the [uniq](#agg_function-uniq) function. +Use the `uniqExact` function if you absolutely need an exact result. Otherwise use the [uniq](#agg_function-uniq) function. -The `uniqExact` function uses more memory than the `uniq`, because the size of the state has unbounded growth as the number of different values increases. +The `uniqExact` function uses more memory than `uniq`, because the size of the state has unbounded growth as the number of different values increases. **Parameters** -Function takes the variable number of parameters. Parameters can be of types: `Tuple`, `Array`, `Date`, `DateTime`, `String`, numeric types. +The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types. **See Also** diff --git a/docs/en/query_language/functions/url_functions.md b/docs/en/query_language/functions/url_functions.md index 93edf705e7e..a21c2cfa0bf 100644 --- a/docs/en/query_language/functions/url_functions.md +++ b/docs/en/query_language/functions/url_functions.md @@ -12,7 +12,42 @@ Returns the protocol. Examples: http, ftp, mailto, magnet... ### domain -Gets the domain. Cut scheme with size less than 16 bytes. +Extracts the host part from URL. + +``` +domain(url) +``` + +**Parameters** + +- `url` — URL. Type: [String](../../data_types/string.md). + + +URL can be specified with or without scheme. Examples: + +``` +svn+ssh://some.svn-hosting.com:80/repo/trunk +some.svn-hosting.com:80/repo/trunk +https://yandex.com/time/ +``` + +**Returned values** + +- Host name. If ClickHouse can parse input string as URL. +- Empty string. If ClickHouse cannot parse input string as URL. + +Type: `String`. + +**Example** + +```sql +SELECT domain('svn+ssh://some.svn-hosting.com:80/repo/trunk') +``` +```text +┌─domain('svn+ssh://some.svn-hosting.com:80/repo/trunk')─┐ +│ some.svn-hosting.com │ +└────────────────────────────────────────────────────────┘ +``` ### domainWithoutWWW @@ -20,7 +55,41 @@ Returns the domain and removes no more than one 'www.' from the beginning of it, ### topLevelDomain -Returns the top-level domain. Example: .ru. +Extracts the the top-level domain from URL. + +``` +topLevelDomain(url) +``` + +**Parameters** + +- `url` — URL. Type: [String](../../data_types/string.md). + +URL can be specified with or without scheme. Examples: + +``` +svn+ssh://some.svn-hosting.com:80/repo/trunk +some.svn-hosting.com:80/repo/trunk +https://yandex.com/time/ +``` + +**Returned values** + +- Domain name. If ClickHouse can parse input string as URL. +- Empty string. If ClickHouse cannot parse input string as URL. + +Type: `String`. + +**Example** + +```sql +SELECT topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk') +``` +```text +┌─topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk')─┐ +│ com │ +└────────────────────────────────────────────────────────────────────┘ +``` ### firstSignificantSubdomain @@ -66,7 +135,7 @@ Returns an array of name strings corresponding to the names of URL parameters. T ### URLHierarchy(URL) -Returns an array containing the URL, truncated at the end by the symbols /,? in the path and query-string. Consecutive separator characters are counted as one. The cut is made in the position after all the consecutive separator characters. +Returns an array containing the URL, truncated at the end by the symbols /,? in the path and query-string. Consecutive separator characters are counted as one. The cut is made in the position after all the consecutive separator characters. ### URLPathHierarchy(URL) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 25804d67322..b5ce6e25be6 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -715,6 +715,19 @@ load_balancing = first_or_random - [insert_quorum](#settings-insert_quorum) - [insert_quorum_timeout](#settings-insert_quorum_timeout) +## count_distinct_implementation {#settings-count_distinct_implementation} + +Задаёт, какая из функций `uniq*` используется при выполнении конструкции [COUNT(DISTINCT ...)](../../query_language/agg_functions/reference.md#agg_function-count). + +Возможные значения: + +- [uniq](../../query_language/agg_functions/reference.md#agg_function-uniq) +- [uniqCombined](../../query_language/agg_functions/reference.md#agg_function-uniqcombined) +- [uniqHLL12](../../query_language/agg_functions/reference.md#agg_function-uniqhll12) +- [uniqExact](../../query_language/agg_functions/reference.md#agg_function-uniqexact) + +Значение по умолчанию — `uniqExact`. + ## max_network_bytes {#settings-max_network_bytes} Ограничивает объем данных (в байтах), который принимается или передается по сети при выполнении запроса. Параметр применяется к каждому отдельному запросу. diff --git a/docs/ru/operations/system_tables.md b/docs/ru/operations/system_tables.md index eb452c8de4e..da58fbe9ca1 100644 --- a/docs/ru/operations/system_tables.md +++ b/docs/ru/operations/system_tables.md @@ -8,10 +8,39 @@ ## system.asynchronous_metrics {#system_tables-asynchronous_metrics} -Содержат метрики, используемые для профилирования и мониторинга. -Обычно отражают количество событий, происходящих в данный момент в системе, или ресурсов, суммарно потребляемых системой. -Пример: количество запросов типа SELECT, исполняемых в текущий момент; количество потребляемой памяти. -`system.asynchronous_metrics` и `system.metrics` отличаются набором и способом вычисления метрик. +Содержит метрики, которые периодически вычисляются в фоновом режиме. Например, объем используемой оперативной памяти. + +Столбцы: + +- `metric` ([String](../data_types/string.md)) — название метрики. +- `value` ([Float64](../data_types/float.md)) — значение метрики. + +**Пример** + +```sql +SELECT * FROM system.asynchronous_metrics LIMIT 10 +``` + +```text +┌─metric──────────────────────────────────┬──────value─┐ +│ jemalloc.background_thread.run_interval │ 0 │ +│ jemalloc.background_thread.num_runs │ 0 │ +│ jemalloc.background_thread.num_threads │ 0 │ +│ jemalloc.retained │ 422551552 │ +│ jemalloc.mapped │ 1682989056 │ +│ jemalloc.resident │ 1656446976 │ +│ jemalloc.metadata_thp │ 0 │ +│ jemalloc.metadata │ 10226856 │ +│ UncompressedCacheCells │ 0 │ +│ MarkCacheFiles │ 0 │ +└─────────────────────────────────────────┴────────────┘ +``` + +**Смотрите также** + +- [Мониторинг](monitoring.md) — основы мониторинга в ClickHouse. +- [system.metrics](#system_tables-metrics) — таблица с мгновенно вычисляемыми метриками. +- [system.events](#system_tables-events) — таблица с количеством произошедших событий. ## system.clusters @@ -19,15 +48,16 @@ Столбцы: ``` -cluster String - имя кластера -shard_num UInt32 - номер шарда в кластере, начиная с 1 -shard_weight UInt32 - относительный вес шарда при записи данных -replica_num UInt32 - номер реплики в шарде, начиная с 1 -host_name String - имя хоста, как прописано в конфиге -host_address String - IP-адрес хоста, полученный из DNS -port UInt16 - порт, на который обращаться для соединения с сервером -user String - имя пользователя, которого использовать для соединения с сервером +cluster String — имя кластера. +shard_num UInt32 — номер шарда в кластере, начиная с 1. +shard_weight UInt32 — относительный вес шарда при записи данных +replica_num UInt32 — номер реплики в шарде, начиная с 1. +host_name String — хост, указанный в конфигурации. +host_address String — IP-адрес хоста, полученный из DNS. +port UInt16 — порт, на который обращаться для соединения с сервером. +user String — имя пользователя, которого использовать для соединения с сервером. ``` + ## system.columns Содержит информацию о столбцах всех таблиц. @@ -72,9 +102,35 @@ default_expression String - выражение для значения по ум ## system.events {#system_tables-events} -Содержит информацию о количестве произошедших в системе событий, для профилирования и мониторинга. -Пример: количество обработанных запросов типа SELECT. -Столбцы: event String - имя события, value UInt64 - количество. +Содержит информацию о количестве событий, произошедших в системе. Например, в таблице можно найти, сколько запросов `SELECT` обработано с момента запуска сервера ClickHouse. + +Столбцы: + +- `event` ([String](../data_types/string.md)) — имя события. +- `value` ([UInt64](../data_types/int_uint.md)) — количество произошедших событий. +- `description` ([String](../data_types/string.md)) — описание события. + +**Пример** + +```sql +SELECT * FROM system.events LIMIT 5 +``` + +```text +┌─event─────────────────────────────────┬─value─┬─description────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ Query │ 12 │ Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. │ +│ SelectQuery │ 8 │ Same as Query, but only for SELECT queries. │ +│ FileOpen │ 73 │ Number of files opened. │ +│ ReadBufferFromFileDescriptorRead │ 155 │ Number of reads (read/pread) from a file descriptor. Does not include sockets. │ +│ ReadBufferFromFileDescriptorReadBytes │ 9931 │ Number of bytes read from file descriptors. If the file is compressed, this will show the compressed data size. │ +└───────────────────────────────────────┴───────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +**Смотрите также** + +- [system.asynchronous_metrics](#system_tables-asynchronous_metrics) — таблица с периодически вычисляемыми метриками. +- [system.metrics](#system_tables-metrics) — таблица с мгновенно вычисляемыми метриками. +- [Мониторинг](monitoring.md) — основы мониторинга в ClickHouse. ## system.functions @@ -123,11 +179,47 @@ default_expression String - выражение для значения по ум ## system.metrics {#system_tables-metrics} +Содержит метрики, которые могут быть рассчитаны мгновенно или имеют текущее значение. Например, число одновременно обрабатываемых запросов или текущее значение задержки реплики. Эта таблица всегда актуальна. + +Столбцы: + +- `metric` ([String](../data_types/string.md)) — название метрики. +- `value` ([Int64](../data_types/int_uint.md)) — значение метрики. +- `description` ([String](../data_types/string.md)) — описание метрики. + +**Пример** + +```sql +SELECT * FROM system.metrics LIMIT 10 +``` + +```text +┌─metric─────────────────────┬─value─┬─description──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ Query │ 1 │ Number of executing queries │ +│ Merge │ 0 │ Number of executing background merges │ +│ PartMutation │ 0 │ Number of mutations (ALTER DELETE/UPDATE) │ +│ ReplicatedFetch │ 0 │ Number of data parts being fetched from replicas │ +│ ReplicatedSend │ 0 │ Number of data parts being sent to replicas │ +│ ReplicatedChecks │ 0 │ Number of data parts checking for consistency │ +│ BackgroundPoolTask │ 0 │ Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches, or replication queue bookkeeping) │ +│ BackgroundSchedulePoolTask │ 0 │ Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc. │ +│ DiskSpaceReservedForMerge │ 0 │ Disk space reserved for currently running background merges. It is slightly more than the total size of currently merging parts. │ +│ DistributedSend │ 0 │ Number of connections to remote servers sending data that was INSERTed into Distributed tables. Both synchronous and asynchronous mode. │ +└────────────────────────────┴───────┴──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +**Смотрите также** + +- [system.asynchronous_metrics](#system_tables-asynchronous_metrics) — таблица с периодически вычисляемыми метриками. +- [system.events](#system_tables-events) — таблица с количеством произошедших событий. +- [Мониторинг](monitoring.md) — основы мониторинга в ClickHouse. + ## system.numbers Таблица содержит один столбец с именем number типа UInt64, содержащим почти все натуральные числа, начиная с нуля. Эту таблицу можно использовать для тестов, а также если вам нужно сделать перебор. Чтения из этой таблицы не распараллеливаются. + ## system.numbers_mt То же самое, что и system.numbers, но чтение распараллеливается. Числа могут возвращаться в произвольном порядке. diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index 6247b74f657..fca564b7a14 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -1,11 +1,71 @@ # Справочник функций -## count() {#agg_function-count} +## count {#agg_function-count} -Считает количество строк. Принимает ноль аргументов, возвращает UInt64. -Не поддерживается синтаксис `COUNT (DISTINCT x)`. Для этого существует агрегатная функция`uniq`. +Вычисляет количество строк или не NULL значений . -Запрос вида `SELECT count() FROM table` не оптимизируется, так как количество записей в таблице нигде не хранится отдельно. Из таблицы будет выбран какой-нибудь достаточно маленький столбец, и будет посчитано количество значений в нём. +ClickHouse поддерживает следующие виды синтаксиса для `count`: + +- `count(expr)` или `COUNT(DISTINCT expr)`. +- `count()` или `COUNT(*)`. Синтаксис `count()` специфичен для ClickHouse. + +**Параметры** + +Функция может принимать: + +- Ноль параметров. +- Одно [выражение](../syntax.md#syntax-expressions). + +**Возвращаемое значение** + +- Если функция вызывается без параметров, она вычисляет количество строк. +- Если передаётся [выражение](../syntax.md#syntax-expressions) , то функция вычисляет количество раз, когда выражение возвращает не NULL. Если выражение возвращает значение типа [Nullable](../../data_types/nullable.md), то результат `count` не становится `Nullable`. Функция возвращает 0, если выражение возвращает `NULL` для всех строк. + +В обоих случаях тип возвращаемого значения [UInt64](../../data_types/int_uint.md). + +**Подробности** + +ClickHouse поддерживает синтаксис `COUNT(DISTINCT ...)`. Поведение этой конструкции зависит от настройки [count_distinct_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation). Она определяет, какая из функций [uniq*](#agg_function-uniq) используется для выполнения операции. По умолчанию — функция [uniqExact](#agg_function-uniqexact). + +Запрос `SELECT count() FROM table` не оптимизирован, поскольку количество записей в таблице не хранится отдельно. Он выбирает небольшой столбец из таблицы и подсчитывает количество значений в нём. + +**Примеры** + +Пример 1: + +```sql +SELECT count() FROM t +``` + +```text +┌─count()─┐ +│ 5 │ +└─────────┘ +``` + +Пример 2: + +```sql +SELECT name, value FROM system.settings WHERE name = 'count_distinct_implementation' +``` + +```text +┌─name──────────────────────────┬─value─────┐ +│ count_distinct_implementation │ uniqExact │ +└───────────────────────────────┴───────────┘ +``` + +```sql +SELECT count(DISTINCT num) FROM t +``` + +```text +┌─uniqExact(num)─┐ +│ 3 │ +└────────────────┘ +``` + +Этот пример показывает, что `count(DISTINCT num)` выполняется с помощью функции `uniqExact` в соответствии со значением настройки `count_distinct_implementation`. ## any(x) {#agg_function-any} @@ -59,7 +119,7 @@ groupBitAnd(expr) **Параметры** -`expr` – Выражение, результат которого имеет тип `UInt*`. +`expr` – выражение, результат которого имеет тип данных `UInt*`. **Возвращаемое значение** @@ -97,12 +157,12 @@ binary decimal Применяет побитовое `ИЛИ` для последовательности чисел. ``` -groupBitOr (expr) +groupBitOr(expr) ``` **Параметры** -`expr` – Выражение, результат которого имеет тип `UInt*`. +`expr` – выражение, результат которого имеет тип данных `UInt*`. **Возвращаемое значение** @@ -145,7 +205,7 @@ groupBitXor(expr) **Параметры** -`expr` – Выражение, результат которого имеет тип `UInt*`. +`expr` – выражение, результат которого имеет тип данных `UInt*`. **Возвращаемое значение** @@ -188,7 +248,7 @@ groupBitmap(expr) **Параметры** -`expr` – выражение, возвращающее тип данных `UInt*`. +`expr` – выражение, результат которого имеет тип данных `UInt*`. **Возвращаемое значение** @@ -227,7 +287,7 @@ num Вычисляет максимум. -## argMin(arg, val) +## argMin(arg, val) {#agg_function-argMin} Вычисляет значение arg при минимальном значении val. Если есть несколько разных значений arg для минимальных значений val, то выдаётся первое попавшееся из таких значений. @@ -247,7 +307,7 @@ SELECT argMin(user, salary) FROM salary └──────────────────────┘ ``` -## argMax(arg, val) +## argMax(arg, val) {#agg_function-argMax} Вычисляет значение arg при максимальном значении val. Если есть несколько разных значений arg для максимальных значений val, то выдаётся первое попавшееся из таких значений. @@ -461,45 +521,143 @@ FROM ( Работает только для чисел. Результат всегда Float64. -## uniq(x) {#agg_function-uniq} +## uniq {#agg_function-uniq} -Приближённо вычисляет количество различных значений аргумента. Работает для чисел, строк, дат, дат-с-временем, для нескольких аргументов и аргументов-кортежей. +Приближённо вычисляет количество различных значений аргумента. -Используется алгоритм типа adaptive sampling: в качестве состояния вычислений используется выборка значений хэшей элементов, размером до 65536. -Алгоритм является очень точным для множеств небольшой кардинальности (до 65536) и очень эффективным по CPU (при расчёте не слишком большого количества таких функций, использование `uniq` почти так же быстро, как использование других агрегатных функций). +``` +uniq(x[, ...]) +``` -Результат детерминирован (не зависит от порядка выполнения запроса). +**Параметры** -Функция обеспечивает высокую точность даже для множеств с высокой кардинальностью (более 10 миллиардов элементов). Рекомендуется для использования по умолчанию. +Функция принимает переменное число входных параметров. Параметры могут быть числовых типов, а также `Tuple`, `Array`, `Date`, `DateTime`, `String`. -## uniqCombined(HLL_precision)(x) +**Возвращаемое значение** -Приближённо вычисляет количество различных значений аргумента. Работает для чисел, строк, дат, дат-с-временем, для нескольких аргументов и аргументов-кортежей. +- Значение с типом данных [UInt64](../../data_types/int_uint.md). -Используется комбинация трёх алгоритмов: массив, хэш-таблица и [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) с таблицей коррекции погрешности. Для небольшого количества различных значений используется массив; при увеличении количества значений, используется хэш таблица, до тех пор, пока её размер меньше размера HyperLogLog структуры. При дальнейшем увеличении количества значений, используется HyperLogLog структура, имеющая фиксированный размер в памяти. +**Детали реализации** -Параметр HLL_precision - логарифм по основанию 2 от количества ячеек в HyperLogLog. Параметер можно не указывать (для этого, опустите первую пару скобок). По-умолчанию - 17. При использовании параметра по-умолчанию, расход памяти в несколько раз меньше, чем у функции `uniq`, а точность в несколько раз выше. Скорость работы чуть ниже, чем у функции `uniq`, но иногда может быть даже выше - в случае распределённых запросов, в которых по сети передаётся большое количество состояний агрегации. Каждая ячейка имеет размер 6 бит, что даёт 96 KiB для размера HyperLogLog структуры. +Функция: -Результат детерминирован (не зависит от порядка выполнения запроса). +- Вычисляет хэш для всех параметров агрегации, а затем использует его в вычислениях. -Функция `uniqCombined` является хорошим выбором по умолчанию для подсчёта количества различных значений, но стоит иметь ввиду что для множеств большой кардинальности (200 миллионов различных элементов и больше) ошибка оценки становится существенно больше расчётной из-за недостаточно хорошего выбора хэш-функции. +- Использует адаптивный алгоритм выборки. В качестве состояния вычисления функция использует выборку хэш-значений элементов размером до 65536. -## uniqHLL12(x) + Этот алгоритм очень точен и очень эффективен по использованию CPU. Если запрос содержит небольшое количество этих функций, использование `uniq` почти так же эффективно, как и использование других агрегатных функций. -Приближённо вычисляет количество различных значений аргумента, используя алгоритм [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). -Используется 212 5-битовых ячеек. Размер состояния чуть больше 2.5 КБ. Результат не очень точный (ошибка до ~10%) для небольших множеств (<10К элементов). Однако, для множеств с большой кардинальностью (10К - 100М) результат имеет ошибку до ~1.6%. Начиная со 100M, ошибка оценки увеличивается и для множеств огромной кардинальности (1B+ элементов) результат будет очень неточным. +- Результат детерминирован (не зависит от порядка выполнения запроса). -Результат детерминирован (не зависит от порядка выполнения запроса). +Эту функцию рекомендуется использовать практически во всех сценариях. -Мы не рекомендуем использовать эту функцию. В большинстве случаев, используйте функцию `uniq` или `uniqCombined`. +**Смотрите также** -## uniqExact(x) +- [uniqCombined](#agg_function-uniqcombined) +- [uniqHLL12](#agg_function-uniqhll12) +- [uniqExact](#agg_function-uniqexact) -Вычисляет количество различных значений аргумента, точно. -Не стоит бояться приближённых расчётов. Поэтому, используйте лучше функцию `uniq`. -Функцию `uniqExact` следует использовать, если вам точно нужен точный результат. +## uniqCombined {#agg_function-uniqcombined} -Функция `uniqExact` расходует больше оперативки, чем функция `uniq`, так как размер состояния неограниченно растёт по мере роста количества различных значений. +Приближённо вычисляет количество различных значений аргумента. + +``` +uniqCombined(HLL_precision)(x[, ...]) +``` + +Функция `uniqCombined` — это хороший выбор для вычисления количества различных значений, однако стоит иметь в виду, что ошибка оценки для больших множеств (более 200 миллионов элементов) будет выше теоретического значения из-за плохого выбора хэш-функции. + +**Параметры** + +Функция принимает переменное число входных параметров. Параметры могут быть числовых типов, а также `Tuple`, `Array`, `Date`, `DateTime`, `String`. + +`HLL_precision` — это логарифм по основанию 2 от числа ячеек в [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Необязательный, можно использовать функцию как `uniqCombined (x [,...])`. Для `HLL_precision` значение по умолчанию — 17, что фактически составляет 96 КБ пространства (2^17 ячеек, 6 бит каждая). + +**Возвращаемое значение** + +- Число типа [UInt64](../../data_types/int_uint.md). + +**Детали реализации** + +Функция: + +- Вычисляет хэш для всех параметров агрегации, а затем использует его в вычислениях. + +- Используется комбинация трёх алгоритмов: массив, хэш-таблица и HyperLogLog с таблицей коррекции погрешности. + + Для небольшого количества различных значений используется массив. Если размер набора больше, используется хэш-таблица. При дальнейшем увеличении количества значений, используется структура HyperLogLog, имеющая фиксированный размер в памяти. + +- Результат детерминирован (не зависит от порядка выполнения запроса). + +По сравнению с функцией [uniq](#agg_function-uniq), `uniqCombined`: + +- Потребляет в несколько раз меньше памяти. +- Вычисляет с в несколько раз более высокой точностью. +- Обычно имеет немного более низкую производительность. В некоторых сценариях `uniqCombined` может показывать более высокую производительность, чем `uniq`, например, в случае распределенных запросов, при которых по сети передаётся большое количество состояний агрегации. + +**Смотрите также** + +- [uniq](#agg_function-uniq) +- [uniqHLL12](#agg_function-uniqhll12) +- [uniqExact](#agg_function-uniqexact) + +## uniqHLL12 {#agg_function-uniqhll12} + +Вычисляет приблизительное число различных значений аргументов, используя алгоритм [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). + +``` +uniqHLL12(x[, ...]) +``` + +**Параметры** + +Функция принимает переменное число входных параметров. Параметры могут быть числовых типов, а также `Tuple`, `Array`, `Date`, `DateTime`, `String`. + +**Возвращаемое значение** + +- Значение хэша с типом данных [UInt64](../../data_types/int_uint.md). + +**Детали реализации** + +Функция: + +- Вычисляет хэш для всех параметров агрегации, а затем использует его в вычислениях. + +- Использует алгоритм HyperLogLog для аппроксимации числа различных значений аргументов. + + Используется 212 5-битовых ячеек. Размер состояния чуть больше 2.5 КБ. Результат не точный (ошибка до ~10%) для небольших множеств (<10K элементов). Однако для множеств большой кардинальности (10K - 100M) результат довольно точен (ошибка до ~1.6%). Начиная с 100M ошибка оценки будет только расти и для множеств огромной кардинальности (1B+ элементов) функция возвращает результат с очень большой неточностью. + +- Результат детерминирован (не зависит от порядка выполнения запроса). + +Мы не рекомендуем использовать эту функцию. В большинстве случаев используйте функцию [uniq](#agg_function-uniq) или [uniqCombined](#agg_function-uniqcombined). + +**Смотрите также** + +- [uniq](#agg_function-uniq) +- [uniqCombined](#agg_function-uniqcombined) +- [uniqExact](#agg_function-uniqexact) + +## uniqExact {#agg_function-uniqexact} + +Вычисляет точное количество различных значений аргументов. + +``` +uniqExact(x[, ...]) +``` + +Функцию `uniqExact` следует использовать, если вам обязательно нужен точный результат. В противном случае используйте функцию [uniq](#agg_function-uniq). + +Функция `uniqExact` расходует больше оперативной памяти, чем функция `uniq`, так как размер состояния неограниченно растёт по мере роста количества различных значений. + +**Параметры** + +Функция принимает переменное число входных параметров. Параметры могут быть числовых типов, а также `Tuple`, `Array`, `Date`, `DateTime`, `String`. + +**Смотрите также** + +- [uniq](#agg_function-uniq) +- [uniqCombined](#agg_function-uniqcombined) +- [uniqHLL12](#agg_function-uniqhll12) ## groupArray(x), groupArray(max_size)(x) @@ -847,3 +1005,4 @@ stochasticLogisticRegression(1.0, 1.0, 10, 'SGD') - [Отличие линейной от логистической регрессии](https://moredez.ru/q/51225972/) [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/agg_functions/reference/) + diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 8ebd9bddc8d..ce8c5801613 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -23,12 +23,10 @@ add_library (common src/getThreadNumber.cpp src/sleep.cpp src/argsToConfig.cpp - src/StackTrace.cpp src/Pipe.cpp src/phdr_cache.cpp include/common/SimpleCache.h - include/common/StackTrace.h include/common/Types.h include/common/DayNum.h include/common/DateLUT.h @@ -68,14 +66,6 @@ add_library (common ${CONFIG_COMMON}) -if (USE_UNWIND) - target_compile_definitions (common PRIVATE USE_UNWIND=1) - target_include_directories (common BEFORE PRIVATE ${UNWIND_INCLUDE_DIR}) - if (NOT USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING) - target_link_libraries (common PRIVATE ${UNWIND_LIBRARY}) - endif () -endif () - # When testing for memory leaks with Valgrind, dont link tcmalloc or jemalloc. if (USE_JEMALLOC) diff --git a/libs/libcommon/include/common/memory.h b/libs/libcommon/include/common/memory.h index d8dced79cfb..58070334ac1 100644 --- a/libs/libcommon/include/common/memory.h +++ b/libs/libcommon/include/common/memory.h @@ -15,10 +15,26 @@ #define USE_JEMALLOC 0 #include #endif +#else +#include #endif -#define ALWAYS_INLINE inline __attribute__((__always_inline__)) -#define NO_INLINE __attribute__((__noinline__)) +// Also defined in Core/Defines.h +#if !defined(ALWAYS_INLINE) +#if defined(_MSC_VER) + #define ALWAYS_INLINE inline __forceinline +#else + #define ALWAYS_INLINE inline __attribute__((__always_inline__)) +#endif +#endif + +#if !defined(NO_INLINE) +#if defined(_MSC_VER) + #define NO_INLINE static __declspec(noinline) +#else + #define NO_INLINE __attribute__((__noinline__)) +#endif +#endif namespace Memory { diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index aa4993acead..16bcb132d37 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/libs/libglibc-compatibility/musl/utimensat.c b/libs/libglibc-compatibility/musl/utimensat.c new file mode 100644 index 00000000000..dce0a3c270f --- /dev/null +++ b/libs/libglibc-compatibility/musl/utimensat.c @@ -0,0 +1,38 @@ +#include +#include +#include +#include +#include "syscall.h" +#include + +int utimensat(int fd, const char *path, const struct timespec times[2], int flags) +{ + int r = __syscall(SYS_utimensat, fd, path, times, flags); +#ifdef SYS_futimesat + if (r != -ENOSYS || flags) return __syscall_ret(r); + struct timeval *tv = 0, tmp[2]; + if (times) { + int i; + tv = tmp; + for (i=0; i<2; i++) { + if (times[i].tv_nsec >= 1000000000ULL) { + if (times[i].tv_nsec == UTIME_NOW && + times[1-i].tv_nsec == UTIME_NOW) { + tv = 0; + break; + } + if (times[i].tv_nsec == UTIME_OMIT) + return __syscall_ret(-ENOSYS); + return __syscall_ret(-EINVAL); + } + tmp[i].tv_sec = times[i].tv_sec; + tmp[i].tv_usec = times[i].tv_nsec / 1000; + } + } + + r = __syscall(SYS_futimesat, fd, path, tv); + if (r != -ENOSYS || fd != AT_FDCWD) return __syscall_ret(r); + r = __syscall(SYS_utimes, path, tv); +#endif + return __syscall_ret(r); +}