diff --git a/base/common/CMakeLists.txt b/base/common/CMakeLists.txt index 9b827cdb468..d98da8f0450 100644 --- a/base/common/CMakeLists.txt +++ b/base/common/CMakeLists.txt @@ -16,6 +16,7 @@ set (SRCS shift10.cpp sleep.cpp terminalColors.cpp + errnoToString.cpp ) if (ENABLE_REPLXX) diff --git a/base/common/ReplxxLineReader.cpp b/base/common/ReplxxLineReader.cpp index 727a44f3e8a..251170ab5c1 100644 --- a/base/common/ReplxxLineReader.cpp +++ b/base/common/ReplxxLineReader.cpp @@ -1,9 +1,11 @@ #include +#include #include #include #include #include +#include namespace { @@ -29,7 +31,29 @@ ReplxxLineReader::ReplxxLineReader( using Replxx = replxx::Replxx; if (!history_file_path.empty()) - rx.history_load(history_file_path); + { + history_file_fd = open(history_file_path.c_str(), O_RDWR); + if (history_file_fd < 0) + { + rx.print("Open of history file failed: %s\n", errnoToString(errno).c_str()); + } + else + { + if (flock(history_file_fd, LOCK_SH)) + { + rx.print("Shared lock of history file failed: %s\n", errnoToString(errno).c_str()); + } + else + { + rx.history_load(history_file_path); + + if (flock(history_file_fd, LOCK_UN)) + { + rx.print("Unlock of history file failed: %s\n", errnoToString(errno).c_str()); + } + } + } + } auto callback = [&suggest] (const String & context, size_t context_size) { @@ -57,8 +81,8 @@ ReplxxLineReader::ReplxxLineReader( ReplxxLineReader::~ReplxxLineReader() { - if (!history_file_path.empty()) - rx.history_save(history_file_path); + if (close(history_file_fd)) + rx.print("Close of history file failed: %s\n", strerror(errno)); } LineReader::InputStatus ReplxxLineReader::readOneLine(const String & prompt) @@ -76,7 +100,20 @@ LineReader::InputStatus ReplxxLineReader::readOneLine(const String & prompt) void ReplxxLineReader::addToHistory(const String & line) { + // locking history file to prevent from inconsistent concurrent changes + bool locked = false; + if (flock(history_file_fd, LOCK_EX)) + rx.print("Lock of history file failed: %s\n", strerror(errno)); + else + locked = true; + rx.history_add(line); + + // flush changes to the disk + rx.history_save(history_file_path); + + if (locked && 0 != flock(history_file_fd, LOCK_UN)) + rx.print("Unlock of history file failed: %s\n", strerror(errno)); } void ReplxxLineReader::enableBracketedPaste() diff --git a/base/common/ReplxxLineReader.h b/base/common/ReplxxLineReader.h index 37e38f9ff34..1fbfd53457b 100644 --- a/base/common/ReplxxLineReader.h +++ b/base/common/ReplxxLineReader.h @@ -25,4 +25,7 @@ private: replxx::Replxx rx; replxx::Replxx::highlighter_callback_t highlighter; + + // used to call flock() to synchronize multiple clients using same history file + int history_file_fd = -1; }; diff --git a/base/common/errnoToString.cpp b/base/common/errnoToString.cpp new file mode 100644 index 00000000000..cdadba2c615 --- /dev/null +++ b/base/common/errnoToString.cpp @@ -0,0 +1,29 @@ +#include "errnoToString.h" + +#include + + +std::string errnoToString(int code, int the_errno) +{ + const size_t buf_size = 128; + char buf[buf_size]; +#ifndef _GNU_SOURCE + int rc = strerror_r(the_errno, buf, buf_size); +#ifdef __APPLE__ + if (rc != 0 && rc != EINVAL) +#else + if (rc != 0) +#endif + { + std::string tmp = std::to_string(code); + const char * code_str = tmp.c_str(); + const char * unknown_message = "Unknown error "; + strcpy(buf, unknown_message); + strcpy(buf + strlen(unknown_message), code_str); + } + return fmt::format("errno: {}, strerror: {}", the_errno, buf); +#else + (void)code; + return fmt::format("errno: {}, strerror: {}", the_errno, strerror_r(the_errno, buf, sizeof(buf))); +#endif +} diff --git a/base/common/errnoToString.h b/base/common/errnoToString.h new file mode 100644 index 00000000000..fd5f81ec2c7 --- /dev/null +++ b/base/common/errnoToString.h @@ -0,0 +1,6 @@ +#pragma once + +#include +#include + +std::string errnoToString(int code, int the_errno = errno); diff --git a/base/common/ya.make b/base/common/ya.make index 6e45b0193c5..d40b1f5abfd 100644 --- a/base/common/ya.make +++ b/base/common/ya.make @@ -47,6 +47,7 @@ SRCS( shift10.cpp sleep.cpp terminalColors.cpp + errnoToString.cpp ) END() diff --git a/cmake/find/hyperscan.cmake b/cmake/find/hyperscan.cmake index 039981fce81..3f65d1eb891 100644 --- a/cmake/find/hyperscan.cmake +++ b/cmake/find/hyperscan.cmake @@ -14,10 +14,12 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/hyperscan/CMakeLists.txt") set (USE_INTERNAL_HYPERSCAN_LIBRARY 0) endif () -if (NOT USE_INTERNAL_HYPERSCAN_LIBRARY) - find_library (HYPERSCAN_LIBRARY hs) - find_path (HYPERSCAN_INCLUDE_DIR NAMES hs/hs.h hs.h PATHS ${HYPERSCAN_INCLUDE_PATHS}) -endif () +# We cannot use OS hyperscan library due to different include path. +# +#if (NOT USE_INTERNAL_HYPERSCAN_LIBRARY) +# find_library (HYPERSCAN_LIBRARY hs) +# find_path (HYPERSCAN_INCLUDE_DIR NAMES hs/hs.h hs.h PATHS ${HYPERSCAN_INCLUDE_PATHS}) +#endif () if (HYPERSCAN_LIBRARY AND HYPERSCAN_INCLUDE_DIR) set (USE_HYPERSCAN 1) diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 786e6620eac..b2e4f76c00c 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -82,8 +82,8 @@ RUN apt-get --allow-unauthenticated update -y \ libcctz-dev \ libldap2-dev \ libsasl2-dev \ - heimdal-multidev - + heimdal-multidev \ + libhyperscan-dev # This symlink required by gcc to find lld compiler diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 5826adb7e9c..611b421d18d 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -41,8 +41,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2 ) ENGINE = MergeTree() +ORDER BY expr [PARTITION BY expr] -[ORDER BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] [TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...] @@ -58,23 +58,27 @@ For a description of parameters, see the [CREATE query description](../../../sql - `ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters. -- `PARTITION BY` — The [partitioning key](custom-partitioning-key.md). +- `ORDER BY` — The sorting key. + + A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`. + + ClickHouse uses the sorting key as a primary key if the primary key is not defined obviously by the `PRIMARY KEY` clause. + + Use the `ORDER BY tuple()` syntax, if you don't need sorting. See [Selecting the Primary Key](#selecting-the-primary-key). + +- `PARTITION BY` — The [partitioning key](custom-partitioning-key.md). Optional. For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](../../../sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. -- `ORDER BY` — The sorting key. - - A tuple of columns or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`. - -- `PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). +- `PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional. By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. -- `SAMPLE BY` — An expression for sampling. +- `SAMPLE BY` — An expression for sampling. Optional. If a sampling expression is used, the primary key must contain it. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. -- `TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). +- `TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. Expression must have one `Date` or `DateTime` column as a result. Example: `TTL date + INTERVAL 1 DAY` @@ -83,7 +87,7 @@ For a description of parameters, see the [CREATE query description](../../../sql For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) -- `SETTINGS` — Additional parameters that control the behavior of the `MergeTree`: +- `SETTINGS` — Additional parameters that control the behavior of the `MergeTree` (optional): - `index_granularity` — Maximum number of data rows between the marks of an index. Default value: 8192. See [Data Storage](#mergetree-data-storage). - `index_granularity_bytes` — Maximum size of data granules in bytes. Default value: 10Mb. To restrict the granule size only by number of rows, set to 0 (not recommended). See [Data Storage](#mergetree-data-storage). @@ -198,6 +202,10 @@ The number of columns in the primary key is not explicitly limited. Depending on A long primary key will negatively affect the insert performance and memory consumption, but extra columns in the primary key do not affect ClickHouse performance during `SELECT` queries. +You can create a table without a primary key using the `ORDER BY tuple()` syntax. In this case, ClickHouse stores data in the order of inserting. If you want to save data order when inserting data by `INSERT ... SELECT` queries, set [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads). + +To select data in the initial order, use [single-threaded](../../../operations/settings/settings.md#settings-max_threads) `SELECT` queries. + ### Choosing a Primary Key that Differs from the Sorting Key {#choosing-a-primary-key-that-differs-from-the-sorting-key} It is possible to specify a primary key (an expression with values that are written in the index file for each mark) that is different from the sorting key (an expression for sorting the rows in data parts). In this case the primary key expression tuple must be a prefix of the sorting key expression tuple. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d70806303d2..8b58c5664b6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Access/AllowedClientHosts.cpp b/src/Access/AllowedClientHosts.cpp index 82372fd8b14..1cee8a2f782 100644 --- a/src/Access/AllowedClientHosts.cpp +++ b/src/Access/AllowedClientHosts.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -44,66 +45,22 @@ namespace return IPSubnet(toIPv6(subnet.getPrefix()), subnet.getMask()); } - - /// Helper function for isAddressOfHost(). - bool isAddressOfHostImpl(const IPAddress & address, const String & host) - { - IPAddress addr_v6 = toIPv6(address); - - /// Resolve by hand, because Poco don't use AI_ALL flag but we need it. - addrinfo * ai_begin = nullptr; - SCOPE_EXIT( - { - if (ai_begin) - freeaddrinfo(ai_begin); - }); - - addrinfo hints; - memset(&hints, 0, sizeof(hints)); - hints.ai_family = AF_UNSPEC; - hints.ai_flags |= AI_V4MAPPED | AI_ALL; - - int err = getaddrinfo(host.c_str(), nullptr, &hints, &ai_begin); - if (err) - throw Exception("Cannot getaddrinfo(" + host + "): " + gai_strerror(err), ErrorCodes::DNS_ERROR); - - for (const addrinfo * ai = ai_begin; ai; ai = ai->ai_next) - { - if (ai->ai_addrlen && ai->ai_addr) - { - if (ai->ai_family == AF_INET) - { - const auto & sin = *reinterpret_cast(ai->ai_addr); - if (addr_v6 == toIPv6(IPAddress(&sin.sin_addr, sizeof(sin.sin_addr)))) - { - return true; - } - } - else if (ai->ai_family == AF_INET6) - { - const auto & sin = *reinterpret_cast(ai->ai_addr); - if (addr_v6 == IPAddress(&sin.sin6_addr, sizeof(sin.sin6_addr), sin.sin6_scope_id)) - { - return true; - } - } - } - } - - return false; - } - - auto & getIsAddressOfHostCache() - { - static SimpleCache cache; - return cache; - } - /// Whether a specified address is one of the addresses of a specified host. bool isAddressOfHost(const IPAddress & address, const String & host) { - /// We need to cache DNS requests. - return getIsAddressOfHostCache()(address, host); + IPAddress addr_v6 = toIPv6(address); + + auto host_addresses = DNSResolver::instance().resolveHostAll(host); + + for (const auto & addr : host_addresses) + { + if (addr.family() == IPAddress::Family::IPv4 && addr_v6 == toIPv6(addr)) + return true; + else if (addr.family() == IPAddress::Family::IPv6 && addr_v6 == addr) + return true; + } + + return false; } /// Helper function for isAddressOfLocalhost(). @@ -147,16 +104,10 @@ namespace return boost::range::find(local_addresses, toIPv6(address)) != local_addresses.end(); } - /// Helper function for getHostByAddress(). - String getHostByAddressImpl(const IPAddress & address) + /// Returns the host name by its address. + String getHostByAddress(const IPAddress & address) { - Poco::Net::SocketAddress sock_addr(address, 0); - - /// Resolve by hand, because Poco library doesn't have such functionality. - char host[1024]; - int err = getnameinfo(sock_addr.addr(), sock_addr.length(), host, sizeof(host), nullptr, 0, NI_NAMEREQD); - if (err) - throw Exception("Cannot getnameinfo(" + address.toString() + "): " + gai_strerror(err), ErrorCodes::DNS_ERROR); + String host = DNSResolver::instance().reverseResolve(address); /// Check that PTR record is resolved back to client address if (!isAddressOfHost(address, host)) @@ -165,19 +116,6 @@ namespace return host; } - auto & getHostByAddressCache() - { - static SimpleCache cache; - return cache; - } - - /// Returns the host name by its address. - String getHostByAddress(const IPAddress & address) - { - /// We need to cache DNS requests. - return getHostByAddressCache()(address); - } - void parseLikePatternIfIPSubnet(const String & pattern, IPSubnet & subnet, IPAddress::Family address_family) { @@ -376,10 +314,4 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const return false; } -void AllowedClientHosts::dropDNSCaches() -{ - getIsAddressOfHostCache().drop(); - getHostByAddressCache().drop(); -} - } diff --git a/src/Access/AllowedClientHosts.h b/src/Access/AllowedClientHosts.h index 4f4d54ce1ac..2baafb2e04a 100644 --- a/src/Access/AllowedClientHosts.h +++ b/src/Access/AllowedClientHosts.h @@ -114,8 +114,6 @@ public: friend bool operator ==(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs); friend bool operator !=(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs) { return !(lhs == rhs); } - static void dropDNSCaches(); - private: std::vector addresses; std::vector subnets; diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index c5b4fd0c585..2f530f2f2de 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -4,20 +4,32 @@ #include #include #include +#include #include #include #include -#include -#include #include #include #include +#include namespace ProfileEvents { extern Event DNSError; } +namespace std +{ +template<> struct hash +{ + size_t operator()(const Poco::Net::IPAddress & address) const noexcept + { + std::string_view addr(static_cast(address.addr()), address.length()); + std::hash hash_impl; + return hash_impl(addr); + } +}; +} namespace DB { @@ -25,6 +37,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int DNS_ERROR; } @@ -76,16 +89,48 @@ static void splitHostAndPort(const std::string & host_and_port, std::string & ou } } -static Poco::Net::IPAddress resolveIPAddressImpl(const std::string & host) +static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) { + Poco::Net::IPAddress ip; + /// NOTE: Poco::Net::DNS::resolveOne(host) doesn't work for IP addresses like 127.0.0.2 - /// Therefore we use SocketAddress constructor with dummy port to resolve IP - return Poco::Net::SocketAddress(host, 0U).host(); + if (Poco::Net::IPAddress::tryParse(host, ip)) + return DNSResolver::IPAddresses(1, ip); + + /// Family: AF_UNSPEC + /// AI_ALL is required for checking if client is allowed to connect from an address + auto flags = Poco::Net::DNS::DNS_HINT_AI_V4MAPPED | Poco::Net::DNS::DNS_HINT_AI_ALL; + /// Do not resolve IPv6 (or IPv4) if no local IPv6 (or IPv4) addresses are configured. + /// It should not affect client address checking, since client cannot connect from IPv6 address + /// if server has no IPv6 addresses. + flags |= Poco::Net::DNS::DNS_HINT_AI_ADDRCONFIG; +#if defined(ARCADIA_BUILD) + auto addresses = Poco::Net::DNS::hostByName(host, &Poco::Net::DNS::DEFAULT_DNS_TIMEOUT, flags).addresses(); +#else + auto addresses = Poco::Net::DNS::hostByName(host, flags).addresses(); +#endif + if (addresses.empty()) + throw Exception("Not found address of host: " + host, ErrorCodes::DNS_ERROR); + + return addresses; +} + +static String reverseResolveImpl(const Poco::Net::IPAddress & address) +{ + Poco::Net::SocketAddress sock_addr(address, 0); + + /// Resolve by hand, because Poco::Net::DNS::hostByAddress(...) does getaddrinfo(...) after getnameinfo(...) + char host[1024]; + int err = getnameinfo(sock_addr.addr(), sock_addr.length(), host, sizeof(host), nullptr, 0, NI_NAMEREQD); + if (err) + throw Exception("Cannot getnameinfo(" + address.toString() + "): " + gai_strerror(err), ErrorCodes::DNS_ERROR); + return host; } struct DNSResolver::Impl { SimpleCache cache_host; + SimpleCache cache_address; std::mutex drop_mutex; std::mutex update_mutex; @@ -95,18 +140,25 @@ struct DNSResolver::Impl /// Store hosts, which was asked to resolve from last update of DNS cache. NameSet new_hosts; + std::unordered_set new_addresses; /// Store all hosts, which was whenever asked to resolve NameSet known_hosts; + std::unordered_set known_addresses; /// If disabled, will not make cache lookups, will resolve addresses manually on each call std::atomic disable_cache{false}; }; -DNSResolver::DNSResolver() : impl(std::make_unique()) {} +DNSResolver::DNSResolver() : impl(std::make_unique()), log(&Poco::Logger::get("DNSResolver")) {} Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host) +{ + return resolveHostAll(host).front(); +} + +DNSResolver::IPAddresses DNSResolver::resolveHostAll(const std::string & host) { if (impl->disable_cache) return resolveIPAddressImpl(host); @@ -125,7 +177,7 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host_an splitHostAndPort(host_and_port, host, port); addToNewHosts(host); - return Poco::Net::SocketAddress(impl->cache_host(host), port); + return Poco::Net::SocketAddress(impl->cache_host(host).front(), port); } Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, UInt16 port) @@ -134,17 +186,29 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, U return Poco::Net::SocketAddress(host, port); addToNewHosts(host); - return Poco::Net::SocketAddress(impl->cache_host(host), port); + return Poco::Net::SocketAddress(impl->cache_host(host).front(), port); +} + +String DNSResolver::reverseResolve(const Poco::Net::IPAddress & address) +{ + if (impl->disable_cache) + return reverseResolveImpl(address); + + addToNewAddresses(address); + return impl->cache_address(address); } void DNSResolver::dropCache() { impl->cache_host.drop(); + impl->cache_address.drop(); std::scoped_lock lock(impl->update_mutex, impl->drop_mutex); impl->known_hosts.clear(); + impl->known_addresses.clear(); impl->new_hosts.clear(); + impl->new_addresses.clear(); impl->host_name.reset(); } @@ -166,34 +230,27 @@ String DNSResolver::getHostName() return *impl->host_name; } -bool DNSResolver::updateCache() +static const String & cacheElemToString(const String & str) { return str; } +static String cacheElemToString(const Poco::Net::IPAddress & addr) { return addr.toString(); } + +template +bool DNSResolver::updateCacheImpl(UpdateF && update_func, ElemsT && elems, const String & log_msg) { - { - std::lock_guard lock(impl->drop_mutex); - for (const auto & host : impl->new_hosts) - impl->known_hosts.insert(host); - impl->new_hosts.clear(); - - impl->host_name.emplace(Poco::Net::DNS::hostName()); - } - - std::lock_guard lock(impl->update_mutex); - bool updated = false; - String lost_hosts; - for (const auto & host : impl->known_hosts) + String lost_elems; + for (const auto & elem : elems) { try { - updated |= updateHost(host); + updated |= (this->*update_func)(elem); } catch (const Poco::Net::NetException &) { ProfileEvents::increment(ProfileEvents::DNSError); - if (!lost_hosts.empty()) - lost_hosts += ", "; - lost_hosts += host; + if (!lost_elems.empty()) + lost_elems += ", "; + lost_elems += cacheElemToString(elem); } catch (...) { @@ -201,12 +258,41 @@ bool DNSResolver::updateCache() } } - if (!lost_hosts.empty()) - LOG_INFO(&Poco::Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts); + if (!lost_elems.empty()) + LOG_INFO(log, log_msg, lost_elems); return updated; } +bool DNSResolver::updateCache() +{ + LOG_DEBUG(log, "Updating DNS cache"); + + { + std::lock_guard lock(impl->drop_mutex); + + for (const auto & host : impl->new_hosts) + impl->known_hosts.insert(host); + impl->new_hosts.clear(); + + for (const auto & address : impl->new_addresses) + impl->known_addresses.insert(address); + impl->new_addresses.clear(); + + impl->host_name.emplace(Poco::Net::DNS::hostName()); + } + + /// FIXME Updating may take a long time becouse we cannot manage timeouts of getaddrinfo(...) and getnameinfo(...). + /// DROP DNS CACHE will wait on update_mutex (possibly while holding drop_mutex) + std::lock_guard lock(impl->update_mutex); + + bool hosts_updated = updateCacheImpl(&DNSResolver::updateHost, impl->known_hosts, "Cached hosts not found: {}"); + updateCacheImpl(&DNSResolver::updateAddress, impl->known_addresses, "Cached addresses not found: {}"); + + LOG_DEBUG(log, "Updated DNS cache"); + return hosts_updated; +} + bool DNSResolver::updateHost(const String & host) { /// Usage of updateHost implies that host is already in cache and there is no extra computations @@ -215,12 +301,25 @@ bool DNSResolver::updateHost(const String & host) return old_value != impl->cache_host(host); } +bool DNSResolver::updateAddress(const Poco::Net::IPAddress & address) +{ + auto old_value = impl->cache_address(address); + impl->cache_address.update(address); + return old_value == impl->cache_address(address); +} + void DNSResolver::addToNewHosts(const String & host) { std::lock_guard lock(impl->drop_mutex); impl->new_hosts.insert(host); } +void DNSResolver::addToNewAddresses(const Poco::Net::IPAddress & address) +{ + std::lock_guard lock(impl->drop_mutex); + impl->new_addresses.insert(address); +} + DNSResolver::~DNSResolver() = default; DNSResolver & DNSResolver::instance() diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index 7dfbe49ab77..7dbc2852d43 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -16,18 +17,26 @@ namespace DB class DNSResolver : private boost::noncopyable { public: + typedef std::vector IPAddresses; + static DNSResolver & instance(); DNSResolver(const DNSResolver &) = delete; - /// Accepts host names like 'example.com' or '127.0.0.1' or '::1' and resolve its IP + /// Accepts host names like 'example.com' or '127.0.0.1' or '::1' and resolves its IP Poco::Net::IPAddress resolveHost(const std::string & host); - /// Accepts host names like 'example.com:port' or '127.0.0.1:port' or '[::1]:port' and resolve its IP and port + /// Accepts host names like 'example.com' or '127.0.0.1' or '::1' and resolves all its IPs + IPAddresses resolveHostAll(const std::string & host); + + /// Accepts host names like 'example.com:port' or '127.0.0.1:port' or '[::1]:port' and resolves its IP and port Poco::Net::SocketAddress resolveAddress(const std::string & host_and_port); Poco::Net::SocketAddress resolveAddress(const std::string & host, UInt16 port); + /// Accepts host IP and resolves its host name + String reverseResolve(const Poco::Net::IPAddress & address); + /// Get this server host name String getHostName(); @@ -44,16 +53,21 @@ public: ~DNSResolver(); private: + template + bool updateCacheImpl(UpdateF && update_func, ElemsT && elems, const String & log_msg); DNSResolver(); struct Impl; std::unique_ptr impl; + Poco::Logger * log; - /// Returns true if IP of host has been changed. + /// Updates cached value and returns true it has been changed. bool updateHost(const String & host); + bool updateAddress(const Poco::Net::IPAddress & address); void addToNewHosts(const String & host); + void addToNewAddresses(const Poco::Net::IPAddress & address); }; } diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index b0c897127c6..f2470ea0406 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -85,31 +86,6 @@ std::string Exception::getStackTraceString() const } -std::string errnoToString(int code, int the_errno) -{ - const size_t buf_size = 128; - char buf[buf_size]; -#ifndef _GNU_SOURCE - int rc = strerror_r(the_errno, buf, buf_size); -#ifdef __APPLE__ - if (rc != 0 && rc != EINVAL) -#else - if (rc != 0) -#endif - { - std::string tmp = std::to_string(code); - const char * code_str = tmp.c_str(); - const char * unknown_message = "Unknown error "; - strcpy(buf, unknown_message); - strcpy(buf + strlen(unknown_message), code_str); - } - return "errno: " + toString(the_errno) + ", strerror: " + std::string(buf); -#else - (void)code; - return "errno: " + toString(the_errno) + ", strerror: " + std::string(strerror_r(the_errno, buf, sizeof(buf))); -#endif -} - void throwFromErrno(const std::string & s, int code, int the_errno) { throw ErrnoException(s + ", " + errnoToString(code, the_errno), code, the_errno); diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 45a1b3d6340..763b90048bb 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -81,7 +81,6 @@ private: using Exceptions = std::vector; -std::string errnoToString(int code, int the_errno = errno); [[noreturn]] void throwFromErrno(const std::string & s, int code, int the_errno = errno); /// Useful to produce some extra information about available space and inodes on device [[noreturn]] void throwFromErrnoWithPath(const std::string & s, const std::string & path, int code, diff --git a/src/Common/PipeFDs.cpp b/src/Common/PipeFDs.cpp index 1f57234534f..d91917c23a4 100644 --- a/src/Common/PipeFDs.cpp +++ b/src/Common/PipeFDs.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index a8b7d51a260..c4c7d21314d 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 1b97ed5689c..53ab2301a0a 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index 758f500e9d2..d228fdb42b6 100644 --- a/src/Common/StatusFile.cpp +++ b/src/Common/StatusFile.cpp @@ -8,6 +8,7 @@ #include #include +#include #include #include diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 845e21e62d8..19f2dc11e85 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -238,7 +238,6 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) { static_assert(is_unsigned_v, "ValueType must be unsigned."); using UnsignedDeltaType = ValueType; - using SignedDeltaType = typename std::make_signed::type; const char * source_end = source + source_size; @@ -287,12 +286,13 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) if (write_spec.data_bits != 0) { const UInt8 sign = reader.readBit(); - SignedDeltaType signed_dd = static_cast(reader.readBits(write_spec.data_bits - 1) + 1); + double_delta = reader.readBits(write_spec.data_bits - 1) + 1; if (sign) { - signed_dd *= -1; + /// It's well defined for unsigned data types. + /// In constrast, it's undefined to do negation of the most negative signed number due to overflow. + double_delta = -double_delta; } - double_delta = static_cast(signed_dd); } const UnsignedDeltaType delta = double_delta + prev_delta; diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 64e6051b8d5..3c7766ba508 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -21,6 +21,8 @@ namespace ErrorCodes UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char * dest) const { + assert(source != nullptr && dest != nullptr); + dest[0] = getMethodByte(); UInt8 header_size = getHeaderSize(); /// Write data from header_size @@ -33,8 +35,9 @@ UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, char * dest) const { - UInt8 header_size = getHeaderSize(); + assert(source != nullptr && dest != nullptr); + UInt8 header_size = getHeaderSize(); if (source_size < header_size) throw Exception("Can't decompress data: the compressed data size (" + toString(source_size) + ", this should include header size) is less than the header size (" + toString(header_size) + ")", ErrorCodes::CORRUPTED_DATA); diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp new file mode 100644 index 00000000000..dc33fc50252 --- /dev/null +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -0,0 +1,1332 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +/// For the expansion of gtest macros. +#if defined(__clang__) + #pragma clang diagnostic ignored "-Wdeprecated" +#elif defined (__GNUC__) && __GNUC__ >= 9 + #pragma GCC diagnostic ignored "-Wdeprecated-copy" +#endif + +#include + +using namespace DB; + +namespace std +{ +template +std::ostream & operator<<(std::ostream & ostr, const std::optional & opt) +{ + if (!opt) + { + return ostr << ""; + } + + return ostr << *opt; +} + +template +std::vector operator+(std::vector && left, std::vector && right) +{ + std::vector result(std::move(left)); + std::move(std::begin(right), std::end(right), std::back_inserter(result)); + + return result; +} + +} + +namespace +{ + +template +struct AsHexStringHelper +{ + const T & container; +}; + +template +std::ostream & operator << (std::ostream & ostr, const AsHexStringHelper & helper) +{ + ostr << std::hex; + for (const auto & e : helper.container) + { + ostr << "\\x" << std::setw(2) << std::setfill('0') << (static_cast(e) & 0xFF); + } + + return ostr; +} + +template +AsHexStringHelper AsHexString(const T & container) +{ + static_assert (sizeof(container[0]) == 1 && std::is_pod>::value, "Only works on containers of byte-size PODs."); + + return AsHexStringHelper{container}; +} + +template +std::string bin(const T & value, size_t bits = sizeof(T)*8) +{ + static const uint8_t MAX_BITS = sizeof(T)*8; + assert(bits <= MAX_BITS); + + return std::bitset(static_cast(value)) + .to_string().substr(MAX_BITS - bits, bits); +} + +template +const char* type_name() +{ +#define MAKE_TYPE_NAME(TYPE) \ + if constexpr (std::is_same_v) return #TYPE + + MAKE_TYPE_NAME(UInt8); + MAKE_TYPE_NAME(UInt16); + MAKE_TYPE_NAME(UInt32); + MAKE_TYPE_NAME(UInt64); + MAKE_TYPE_NAME(Int8); + MAKE_TYPE_NAME(Int16); + MAKE_TYPE_NAME(Int32); + MAKE_TYPE_NAME(Int64); + MAKE_TYPE_NAME(Float32); + MAKE_TYPE_NAME(Float64); + +#undef MAKE_TYPE_NAME + + return typeid(T).name(); +} + +template +DataTypePtr makeDataType() +{ +#define MAKE_DATA_TYPE(TYPE) \ + if constexpr (std::is_same_v) return std::make_shared() + + MAKE_DATA_TYPE(UInt8); + MAKE_DATA_TYPE(UInt16); + MAKE_DATA_TYPE(UInt32); + MAKE_DATA_TYPE(UInt64); + MAKE_DATA_TYPE(Int8); + MAKE_DATA_TYPE(Int16); + MAKE_DATA_TYPE(Int32); + MAKE_DATA_TYPE(Int64); + MAKE_DATA_TYPE(Float32); + MAKE_DATA_TYPE(Float64); + +#undef MAKE_DATA_TYPE + + assert(false && "unknown datatype"); + return nullptr; +} + +template +class BinaryDataAsSequenceOfValuesIterator +{ + const Container & container; + const void * data; + const void * data_end; + + T current_value; + +public: + using Self = BinaryDataAsSequenceOfValuesIterator; + + explicit BinaryDataAsSequenceOfValuesIterator(const Container & container_) + : container(container_), + data(container.data()), + data_end(container.data() + container.size()), + current_value(T{}) + { + static_assert(sizeof(container[0]) == 1 && std::is_pod>::value, "Only works on containers of byte-size PODs."); + read(); + } + + const T & operator*() const + { + return current_value; + } + + size_t itemsLeft() const + { + return reinterpret_cast(data_end) - reinterpret_cast(data); + } + + Self & operator++() + { + read(); + return *this; + } + + explicit operator bool() const + { + return itemsLeft() > 0; + } + +private: + void read() + { + if (!*this) + { + throw std::runtime_error("No more data to read"); + } + + current_value = unalignedLoad(data); + data = reinterpret_cast(data) + sizeof(T); + } +}; + +template +BinaryDataAsSequenceOfValuesIterator AsSequenceOf(const Container & container) +{ + return BinaryDataAsSequenceOfValuesIterator(container); +} + +template +::testing::AssertionResult EqualByteContainersAs(const ContainerLeft & left, const ContainerRight & right) +{ + static_assert(sizeof(typename ContainerLeft::value_type) == 1, "Expected byte-container"); + static_assert(sizeof(typename ContainerRight::value_type) == 1, "Expected byte-container"); + + ::testing::AssertionResult result = ::testing::AssertionSuccess(); + + const auto l_size = left.size() / sizeof(T); + const auto r_size = right.size() / sizeof(T); + const auto size = std::min(l_size, r_size); + + if (l_size != r_size) + { + result = ::testing::AssertionFailure() << "size mismatch, expected: " << l_size << " got:" << r_size; + } + if (l_size == 0 || r_size == 0) + { + return result; + } + + auto l = AsSequenceOf(left); + auto r = AsSequenceOf(right); + + static constexpr auto MAX_MISMATCHING_ITEMS = 5; + int mismatching_items = 0; + size_t i = 0; + + while (l && r) + { + const auto left_value = *l; + const auto right_value = *r; + ++l; + ++r; + ++i; + + if (left_value != right_value) + { + if (result) + { + result = ::testing::AssertionFailure(); + } + + if (++mismatching_items <= MAX_MISMATCHING_ITEMS) + { + result << "\nmismatching " << sizeof(T) << "-byte item #" << i + << "\nexpected: " << bin(left_value) << " (0x" << std::hex << left_value << ")" + << "\ngot : " << bin(right_value) << " (0x" << std::hex << right_value << ")"; + if (mismatching_items == MAX_MISMATCHING_ITEMS) + { + result << "\n..." << std::endl; + } + } + } + } + if (mismatching_items > 0) + { + result << "total mismatching items:" << mismatching_items << " of " << size; + } + + return result; +} + +template +::testing::AssertionResult EqualByteContainers(uint8_t element_size, const ContainerLeft & left, const ContainerRight & right) +{ + switch (element_size) + { + case 1: + return EqualByteContainersAs(left, right); + break; + case 2: + return EqualByteContainersAs(left, right); + break; + case 4: + return EqualByteContainersAs(left, right); + break; + case 8: + return EqualByteContainersAs(left, right); + break; + default: + assert(false && "Invalid element_size"); + return ::testing::AssertionFailure() << "Invalid element_size: " << element_size; + } +} + +struct Codec +{ + std::string codec_statement; + std::optional expected_compression_ratio; + + explicit Codec(std::string codec_statement_, std::optional expected_compression_ratio_ = std::nullopt) + : codec_statement(std::move(codec_statement_)), + expected_compression_ratio(expected_compression_ratio_) + {} +}; + + +struct CodecTestSequence +{ + std::string name; + std::vector serialized_data; + DataTypePtr data_type; + + CodecTestSequence(std::string name_, std::vector serialized_data_, DataTypePtr data_type_) + : name(name_), + serialized_data(serialized_data_), + data_type(data_type_) + {} + + CodecTestSequence & append(const CodecTestSequence & other) + { + assert(data_type->equals(*other.data_type)); + + serialized_data.insert(serialized_data.end(), other.serialized_data.begin(), other.serialized_data.end()); + if (!name.empty()) + name += " + "; + name += other.name; + + return *this; + } +}; + +CodecTestSequence operator+(CodecTestSequence && left, const CodecTestSequence & right) +{ + return left.append(right); +} + +template +CodecTestSequence operator*(CodecTestSequence && left, T times) +{ + std::vector data(std::move(left.serialized_data)); + const size_t initial_size = data.size(); + const size_t final_size = initial_size * times; + + data.reserve(final_size); + + for (T i = 0; i < times; ++i) + { + data.insert(data.end(), data.begin(), data.begin() + initial_size); + } + + return CodecTestSequence{ + left.name + " x " + std::to_string(times), + std::move(data), + std::move(left.data_type) + }; +} + +std::ostream & operator<<(std::ostream & ostr, const Codec & codec) +{ + return ostr << "Codec{" + << "name: " << codec.codec_statement + << ", expected_compression_ratio: " << codec.expected_compression_ratio + << "}"; +} + +std::ostream & operator<<(std::ostream & ostr, const CodecTestSequence & seq) +{ + return ostr << "CodecTestSequence{" + << "name: " << seq.name + << ", type name: " << seq.data_type->getName() + << ", data size: " << seq.serialized_data.size() << " bytes" + << "}"; +} + +template +CodecTestSequence makeSeq(Args && ... args) +{ + std::initializer_list vals{static_cast(args)...}; + std::vector data(sizeof(T) * std::size(vals)); + + char * write_pos = data.data(); + for (const auto & v : vals) + { + unalignedStore(write_pos, v); + write_pos += sizeof(v); + } + + return CodecTestSequence{ + (boost::format("%1% values of %2%") % std::size(vals) % type_name()).str(), + std::move(data), + makeDataType() + }; +} + +template +CodecTestSequence generateSeq(Generator gen, const char* gen_name, B Begin = 0, E End = 10000) +{ + const auto direction = std::signbit(End - Begin) ? -1 : 1; + std::vector data(sizeof(T) * (End - Begin)); + char * write_pos = data.data(); + + for (auto i = Begin; i < End; i += direction) + { + const T v = gen(static_cast(i)); + + unalignedStore(write_pos, v); + write_pos += sizeof(v); + } + + return CodecTestSequence{ + (boost::format("%1% values of %2% from %3%") % (End - Begin) % type_name() % gen_name).str(), + std::move(data), + makeDataType() + }; +} + +struct NoOpTimer +{ + void start() {} + void report(const char*) {} +}; + +struct StopwatchTimer +{ + explicit StopwatchTimer(clockid_t clock_type, size_t estimated_marks = 32) + : stopwatch(clock_type) + { + results.reserve(estimated_marks); + } + + void start() + { + stopwatch.restart(); + } + + void report(const char * mark) + { + results.emplace_back(mark, stopwatch.elapsed()); + } + + void stop() + { + stopwatch.stop(); + } + + const std::vector> & getResults() const + { + return results; + } + +private: + Stopwatch stopwatch; + std::vector> results; +}; + +CompressionCodecPtr makeCodec(const std::string & codec_string, const DataTypePtr data_type) +{ + const std::string codec_statement = "(" + codec_string + ")"; + Tokens tokens(codec_statement.begin().base(), codec_statement.end().base()); + IParser::Pos token_iterator(tokens, 0); + + Expected expected; + ASTPtr codec_ast; + ParserCodec parser; + + parser.parse(token_iterator, codec_ast, expected); + + return CompressionCodecFactory::instance().get(codec_ast, data_type, false); +} + +template +void testTranscoding(Timer & timer, ICompressionCodec & codec, const CodecTestSequence & test_sequence, std::optional expected_compression_ratio = std::optional{}) +{ + const auto & source_data = test_sequence.serialized_data; + + const UInt32 encoded_max_size = codec.getCompressedReserveSize(source_data.size()); + PODArray encoded(encoded_max_size); + + timer.start(); + + assert(source_data.data() != nullptr); // Codec assumes that source buffer is not null. + const UInt32 encoded_size = codec.compress(source_data.data(), source_data.size(), encoded.data()); + timer.report("encoding"); + + encoded.resize(encoded_size); + + PODArray decoded(source_data.size()); + + timer.start(); + const UInt32 decoded_size = codec.decompress(encoded.data(), encoded.size(), decoded.data()); + timer.report("decoding"); + + decoded.resize(decoded_size); + + ASSERT_TRUE(EqualByteContainers(test_sequence.data_type->getSizeOfValueInMemory(), source_data, decoded)); + + const auto header_size = codec.getHeaderSize(); + const auto compression_ratio = (encoded_size - header_size) / (source_data.size() * 1.0); + + if (expected_compression_ratio) + { + ASSERT_LE(compression_ratio, *expected_compression_ratio) + << "\n\tdecoded size: " << source_data.size() + << "\n\tencoded size: " << encoded_size + << "(no header: " << encoded_size - header_size << ")"; + } +} + +class CodecTest : public ::testing::TestWithParam> +{ +public: + enum MakeCodecParam + { + CODEC_WITH_DATA_TYPE, + CODEC_WITHOUT_DATA_TYPE, + }; + + static CompressionCodecPtr makeCodec(MakeCodecParam with_data_type) + { + const auto & codec_string = std::get<0>(GetParam()).codec_statement; + const auto & data_type = with_data_type == CODEC_WITH_DATA_TYPE ? std::get<1>(GetParam()).data_type : nullptr; + + return ::makeCodec(codec_string, data_type); + } + + static void testTranscoding(ICompressionCodec & codec) + { + NoOpTimer timer; + ::testTranscoding(timer, codec, std::get<1>(GetParam()), std::get<0>(GetParam()).expected_compression_ratio); + } +}; + +TEST_P(CodecTest, TranscodingWithDataType) +{ + const auto codec = makeCodec(CODEC_WITH_DATA_TYPE); + testTranscoding(*codec); +} + +TEST_P(CodecTest, TranscodingWithoutDataType) +{ + const auto codec = makeCodec(CODEC_WITHOUT_DATA_TYPE); + testTranscoding(*codec); +} + +// Param is tuple-of-tuple to simplify instantiating with values, since typically group of cases test only one codec. +class CodecTestCompatibility : public ::testing::TestWithParam>> +{}; + +// Check that iput sequence when encoded matches the encoded string binary. +TEST_P(CodecTestCompatibility, Encoding) +{ + const auto & codec_spec = std::get<0>(GetParam()); + const auto & [data_sequence, expected] = std::get<1>(GetParam()); + const auto codec = makeCodec(codec_spec.codec_statement, data_sequence.data_type); + + const auto & source_data = data_sequence.serialized_data; + + // Just encode the data with codec + const UInt32 encoded_max_size = codec->getCompressedReserveSize(source_data.size()); + PODArray encoded(encoded_max_size); + + const UInt32 encoded_size = codec->compress(source_data.data(), source_data.size(), encoded.data()); + encoded.resize(encoded_size); + SCOPED_TRACE(::testing::Message("encoded: ") << AsHexString(encoded)); + + ASSERT_TRUE(EqualByteContainersAs(expected, encoded)); +} + +// Check that binary string is exactly decoded into input sequence. +TEST_P(CodecTestCompatibility, Decoding) +{ + const auto & codec_spec = std::get<0>(GetParam()); + const auto & [expected, encoded_data] = std::get<1>(GetParam()); + const auto codec = makeCodec(codec_spec.codec_statement, expected.data_type); + + PODArray decoded(expected.serialized_data.size()); + const UInt32 decoded_size = codec->decompress(encoded_data.c_str(), encoded_data.size(), decoded.data()); + decoded.resize(decoded_size); + + ASSERT_TRUE(EqualByteContainers(expected.data_type->getSizeOfValueInMemory(), expected.serialized_data, decoded)); +} + +class CodecTestPerformance : public ::testing::TestWithParam> +{}; + +TEST_P(CodecTestPerformance, TranscodingWithDataType) +{ + const auto & [codec_spec, test_seq] = GetParam(); + const auto codec = ::makeCodec(codec_spec.codec_statement, test_seq.data_type); + + const auto runs = 10; + std::map> results; + + for (size_t i = 0; i < runs; ++i) + { + StopwatchTimer timer{CLOCK_THREAD_CPUTIME_ID}; + ::testTranscoding(timer, *codec, test_seq); + timer.stop(); + + for (const auto & [label, value] : timer.getResults()) + { + results[label].push_back(value); + } + } + + auto compute_mean_and_stddev = [](const auto & values) + { + double mean{}; + + if (values.size() < 2) + return std::make_tuple(mean, double{}); + + using ValueType = typename std::decay_t::value_type; + std::vector tmp_v(std::begin(values), std::end(values)); + std::sort(tmp_v.begin(), tmp_v.end()); + + // remove min and max + tmp_v.erase(tmp_v.begin()); + tmp_v.erase(tmp_v.end() - 1); + + for (const auto & v : tmp_v) + { + mean += v; + } + + mean = mean / tmp_v.size(); + double std_dev = 0.0; + for (const auto & v : tmp_v) + { + const auto d = (v - mean); + std_dev += (d * d); + } + std_dev = std::sqrt(std_dev / tmp_v.size()); + + return std::make_tuple(mean, std_dev); + }; + + std::cerr << codec_spec.codec_statement + << " " << test_seq.data_type->getName() + << " (" << test_seq.serialized_data.size() << " bytes, " + << std::hex << CityHash_v1_0_2::CityHash64(test_seq.serialized_data.data(), test_seq.serialized_data.size()) << std::dec + << ", average of " << runs << " runs, μs)"; + + for (const auto & k : {"encoding", "decoding"}) + { + const auto & values = results[k]; + const auto & [mean, std_dev] = compute_mean_and_stddev(values); + // Ensure that Coefficient of variation is reasonably low, otherwise these numbers are meaningless + EXPECT_GT(0.05, std_dev / mean); + std::cerr << "\t" << std::fixed << std::setprecision(1) << mean / 1000.0; + } + + std::cerr << std::endl; +} + +/////////////////////////////////////////////////////////////////////////////////////////////////// +// Here we use generators to produce test payload for codecs. +// Generator is a callable that can produce infinite number of values, +// output value MUST be of the same type as input value. +/////////////////////////////////////////////////////////////////////////////////////////////////// + +auto SameValueGenerator = [](auto value) +{ + return [=](auto i) + { + return static_cast(value); + }; +}; + +auto SequentialGenerator = [](auto stride = 1) +{ + return [=](auto i) + { + using ValueType = decltype(i); + return static_cast(stride * i); + }; +}; + +// Generator that helps debugging output of other generators +// by logging every output value alongside iteration index and input. +//auto LoggingProxyGenerator = [](auto other_generator, const char * name, std::ostream & ostr, const int limit = std::numeric_limits::max()) +//{ +// ostr << "\n\nValues from " << name << ":\n"; +// auto count = std::make_shared(0); +// return [&, count](auto i) +// { +// using ValueType = decltype(i); +// const auto ret = static_cast(other_generator(i)); +// if (++(*count) < limit) +// { +// ostr << "\t" << *count << " : " << i << " => " << ret << "\n"; +// } + +// return ret; +// }; +//}; + +template +using uniform_distribution = +typename std::conditional_t, std::uniform_real_distribution, + typename std::conditional_t, std::uniform_int_distribution, void>>; + + +template +struct MonotonicGenerator // NOLINT +{ + explicit MonotonicGenerator(T stride_ = 1, T max_step = 10) // NOLINT + : prev_value(0), + stride(stride_), + random_engine(0), + distribution(0, max_step) + {} + + template + U operator()(U) + { + prev_value = prev_value + stride * distribution(random_engine); + return static_cast(prev_value); + } + +private: + T prev_value; + const T stride; + std::default_random_engine random_engine; + uniform_distribution distribution; +}; + +template +struct RandomGenerator +{ + explicit RandomGenerator(T seed = 0, T value_min = std::numeric_limits::min(), T value_max = std::numeric_limits::max()) + : random_engine(seed), + distribution(value_min, value_max) + { + } + + template + U operator()(U) + { + return static_cast(distribution(random_engine)); + } + +private: + std::default_random_engine random_engine; + uniform_distribution distribution; +}; + +auto RandomishGenerator = [](auto i) +{ + using T = decltype(i); + double sin_value = sin(static_cast(i * i)) * i; + if (sin_value < std::numeric_limits::lowest() || sin_value > std::numeric_limits::max()) + return T{}; + return T(sin_value); +}; + +auto MinMaxGenerator = []() +{ + return [step = 0](auto i) mutable + { + if (step++ % 2 == 0) + { + return std::numeric_limits::min(); + } + else + { + return std::numeric_limits::max(); + } + }; +}; + +// Fill dest value with 0x00 or 0xFF +auto FFand0Generator = []() +{ + return [step = 0](auto i) mutable + { + decltype(i) result; + if (step++ % 2 == 0) + { + memset(&result, 0, sizeof(result)); + } + else + { + memset(&result, 0xFF, sizeof(result)); + } + + return result; + }; +}; + + +// Makes many sequences with generator, first sequence length is 0, second is 1..., third is 2 up to `sequences_count`. +template +std::vector generatePyramidOfSequences(const size_t sequences_count, Generator && generator, const char* generator_name) +{ + std::vector sequences; + sequences.reserve(sequences_count); + + // Don't test against sequence of size 0, since it causes a nullptr source buffer as codec input and produces an error. + // sequences.push_back(makeSeq()); // sequence of size 0 + for (size_t i = 1; i < sequences_count; ++i) + { + std::string name = generator_name + std::string(" from 0 to ") + std::to_string(i); + sequences.push_back(generateSeq(std::forward(generator), name.c_str(), 0, i)); + } + + return sequences; +}; + +// helper macro to produce human-friendly sequence name from generator +#define G(generator) generator, #generator + +const auto DefaultCodecsToTest = ::testing::Values( + Codec("DoubleDelta"), + Codec("DoubleDelta, LZ4"), + Codec("DoubleDelta, ZSTD"), + Codec("Gorilla"), + Codec("Gorilla, LZ4"), + Codec("Gorilla, ZSTD") +); + +/////////////////////////////////////////////////////////////////////////////////////////////////// +// test cases +/////////////////////////////////////////////////////////////////////////////////////////////////// + +INSTANTIATE_TEST_SUITE_P(Simple, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + makeSeq(1, 2, 3, 5, 7, 11, 13, 17, 23, 29, 31, 37, 41, 43, 47, 53, 59, 61, 67, 71, 73, 79, 83, 89, 97) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(SmallSequences, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::ValuesIn( + generatePyramidOfSequences(42, G(SequentialGenerator(1))) + + generatePyramidOfSequences(42, G(SequentialGenerator(1))) + + generatePyramidOfSequences(42, G(SequentialGenerator(1))) + + generatePyramidOfSequences(42, G(SequentialGenerator(1))) + + generatePyramidOfSequences(42, G(SequentialGenerator(1))) + + generatePyramidOfSequences(42, G(SequentialGenerator(1))) + + generatePyramidOfSequences(42, G(SequentialGenerator(1))) + + generatePyramidOfSequences(42, G(SequentialGenerator(1))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(Mixed, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001), + generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001), + generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001), + generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001), + generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001), + generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001), + generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001), + generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(SameValueInt, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(SameValueGenerator(1000))), + generateSeq(G(SameValueGenerator(1000))), + generateSeq(G(SameValueGenerator(1000))), + generateSeq(G(SameValueGenerator(1000))), + generateSeq(G(SameValueGenerator(1000))), + generateSeq(G(SameValueGenerator(1000))), + generateSeq(G(SameValueGenerator(1000))), + generateSeq(G(SameValueGenerator(1000))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(SameNegativeValueInt, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(SameValueGenerator(-1000))), + generateSeq(G(SameValueGenerator(-1000))), + generateSeq(G(SameValueGenerator(-1000))), + generateSeq(G(SameValueGenerator(-1000))), + generateSeq(G(SameValueGenerator(-1000))), + generateSeq(G(SameValueGenerator(-1000))), + generateSeq(G(SameValueGenerator(-1000))), + generateSeq(G(SameValueGenerator(-1000))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(SameValueFloat, + CodecTest, + ::testing::Combine( + ::testing::Values( + Codec("Gorilla"), + Codec("Gorilla, LZ4") + ), + ::testing::Values( + generateSeq(G(SameValueGenerator(M_E))), + generateSeq(G(SameValueGenerator(M_E))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(SameNegativeValueFloat, + CodecTest, + ::testing::Combine( + ::testing::Values( + Codec("Gorilla"), + Codec("Gorilla, LZ4") + ), + ::testing::Values( + generateSeq(G(SameValueGenerator(-1 * M_E))), + generateSeq(G(SameValueGenerator(-1 * M_E))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(SequentialInt, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(SequentialGenerator(1))), + generateSeq(G(SequentialGenerator(1))), + generateSeq(G(SequentialGenerator(1))), + generateSeq(G(SequentialGenerator(1))), + generateSeq(G(SequentialGenerator(1))), + generateSeq(G(SequentialGenerator(1))), + generateSeq(G(SequentialGenerator(1))), + generateSeq(G(SequentialGenerator(1))) + ) + ) +); + +// -1, -2, -3, ... etc for signed +// 0xFF, 0xFE, 0xFD, ... for unsigned +INSTANTIATE_TEST_SUITE_P(SequentialReverseInt, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(SequentialGenerator(-1))), + generateSeq(G(SequentialGenerator(-1))), + generateSeq(G(SequentialGenerator(-1))), + generateSeq(G(SequentialGenerator(-1))), + generateSeq(G(SequentialGenerator(-1))), + generateSeq(G(SequentialGenerator(-1))), + generateSeq(G(SequentialGenerator(-1))), + generateSeq(G(SequentialGenerator(-1))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(SequentialFloat, + CodecTest, + ::testing::Combine( + ::testing::Values( + Codec("Gorilla"), + Codec("Gorilla, LZ4") + ), + ::testing::Values( + generateSeq(G(SequentialGenerator(M_E))), + generateSeq(G(SequentialGenerator(M_E))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(SequentialReverseFloat, + CodecTest, + ::testing::Combine( + ::testing::Values( + Codec("Gorilla"), + Codec("Gorilla, LZ4") + ), + ::testing::Values( + generateSeq(G(SequentialGenerator(-1 * M_E))), + generateSeq(G(SequentialGenerator(-1 * M_E))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(MonotonicInt, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(MonotonicGenerator(1, 5))), + generateSeq(G(MonotonicGenerator(1, 5))), + generateSeq(G(MonotonicGenerator(1, 5))), + generateSeq(G(MonotonicGenerator(1, 5))), + generateSeq(G(MonotonicGenerator(1, 5))), + generateSeq(G(MonotonicGenerator(1, 5))), + generateSeq(G(MonotonicGenerator(1, 5))), + generateSeq(G(MonotonicGenerator(1, 5))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(MonotonicReverseInt, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(MonotonicGenerator(-1, 5))), + generateSeq(G(MonotonicGenerator(-1, 5))), + generateSeq(G(MonotonicGenerator(-1, 5))), + generateSeq(G(MonotonicGenerator(-1, 5))), + generateSeq(G(MonotonicGenerator(-1, 5))), + generateSeq(G(MonotonicGenerator(-1, 5))), + generateSeq(G(MonotonicGenerator(-1, 5))), + generateSeq(G(MonotonicGenerator(-1, 5))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(MonotonicFloat, + CodecTest, + ::testing::Combine( + ::testing::Values( + Codec("Gorilla") + ), + ::testing::Values( + generateSeq(G(MonotonicGenerator(M_E, 5))), + generateSeq(G(MonotonicGenerator(M_E, 5))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(MonotonicReverseFloat, + CodecTest, + ::testing::Combine( + ::testing::Values( + Codec("Gorilla") + ), + ::testing::Values( + generateSeq(G(MonotonicGenerator(-1 * M_E, 5))), + generateSeq(G(MonotonicGenerator(-1 * M_E, 5))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(RandomInt, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(RandomGenerator(0))), + generateSeq(G(RandomGenerator(0))), + generateSeq(G(RandomGenerator(0, 0, 1000'000'000))), + generateSeq(G(RandomGenerator(0, 0, 1000'000'000))) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(RandomishInt, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(RandomishGenerator)), + generateSeq(G(RandomishGenerator)), + generateSeq(G(RandomishGenerator)), + generateSeq(G(RandomishGenerator)), + generateSeq(G(RandomishGenerator)), + generateSeq(G(RandomishGenerator)) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(RandomishFloat, + CodecTest, + ::testing::Combine( + DefaultCodecsToTest, + ::testing::Values( + generateSeq(G(RandomishGenerator)), + generateSeq(G(RandomishGenerator)) + ) + ) +); + +// Double delta overflow case, deltas are out of bounds for target type +INSTANTIATE_TEST_SUITE_P(OverflowInt, + CodecTest, + ::testing::Combine( + ::testing::Values( + Codec("DoubleDelta", 1.2), + Codec("DoubleDelta, LZ4", 1.0) + ), + ::testing::Values( + generateSeq(G(MinMaxGenerator())), + generateSeq(G(MinMaxGenerator())), + generateSeq(G(MinMaxGenerator())), + generateSeq(G(MinMaxGenerator())) + ) + ) +); + +INSTANTIATE_TEST_SUITE_P(OverflowFloat, + CodecTest, + ::testing::Combine( + ::testing::Values( + Codec("Gorilla", 1.1), + Codec("Gorilla, LZ4", 1.0) + ), + ::testing::Values( + generateSeq(G(MinMaxGenerator())), + generateSeq(G(MinMaxGenerator())), + generateSeq(G(FFand0Generator())), + generateSeq(G(FFand0Generator())) + ) + ) +); + +template +auto DDCompatibilityTestSequence() +{ + // Generates sequences with double delta in given range. + auto dd_generator = [prev_delta = static_cast(0), prev = static_cast(0)](auto dd) mutable + { + const auto curr = dd + prev + prev_delta; + prev = curr; + prev_delta = dd + prev_delta; + return curr; + }; + + auto ret = generateSeq(G(SameValueGenerator(42)), 0, 3); + + // These values are from DoubleDelta paper (and implementation) and represent points at which DD encoded length is changed. + // DD value less that this point is encoded in shorter binary form (bigger - longer binary). + const Int64 dd_corner_points[] = {-63, 64, -255, 256, -2047, 2048, std::numeric_limits::min(), std::numeric_limits::max()}; + for (const auto & p : dd_corner_points) + { + if (std::abs(p) > std::numeric_limits::max()) + { + break; + } + + // - 4 is to allow DD value to settle before transitioning through important point, + // since DD depends on 2 previous values of data, + 2 is arbitrary. + ret.append(generateSeq(G(dd_generator), p - 4, p + 2)); + } + + return ret; +} + +#define BIN_STR(x) std::string{x, sizeof(x) - 1} + +INSTANTIATE_TEST_SUITE_P(DoubleDelta, + CodecTestCompatibility, + ::testing::Combine( + ::testing::Values(Codec("DoubleDelta")), + ::testing::ValuesIn(std::initializer_list>{ + { + DDCompatibilityTestSequence(), + BIN_STR("\x94\x21\x00\x00\x00\x0f\x00\x00\x00\x01\x00\x0f\x00\x00\x00\x2a\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xb1\xaa\xf4\xf6\x7d\x87\xf8\x80") + }, + { + DDCompatibilityTestSequence(), + BIN_STR("\x94\x27\x00\x00\x00\x15\x00\x00\x00\x01\x00\x15\x00\x00\x00\x2a\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xb1\xaa\xf4\xf6\x7d\x87\xf8\x81\x8e\xd0\xca\x02\x01\x01") + }, + { + DDCompatibilityTestSequence(), + BIN_STR("\x94\x70\x00\x00\x00\x4e\x00\x00\x00\x02\x00\x27\x00\x00\x00\x2a\x00\x00\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xbc\xe3\x5d\xa3\xd3\xd9\xf6\x1f\xe2\x07\x7c\x47\x20\x67\x48\x07\x47\xff\x47\xf6\xfe\xf8\x00\x00\x70\x6b\xd0\x00\x02\x83\xd9\xfb\x9f\xdc\x1f\xfc\x20\x1e\x80\x00\x22\xc8\xf0\x00\x00\x66\x67\xa0\x00\x02\x00\x3d\x00\x00\x0f\xff\xe8\x00\x00\x7f\xee\xff\xdf\x40\x00\x0f\xf2\x78\x00\x01\x7f\x83\x9f\xf7\x9f\xfb\xc0\x00\x00\xff\xfe\x00\x00\x08\x00") + }, + { + DDCompatibilityTestSequence(), + BIN_STR("\x94\x70\x00\x00\x00\x4e\x00\x00\x00\x02\x00\x27\x00\x00\x00\x2a\x00\x00\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xbc\xe3\x5d\xa3\xd3\xd9\xf6\x1f\xe2\x07\x7c\x47\x20\x67\x48\x07\x47\xff\x47\xf6\xfe\xf8\x00\x00\x70\x6b\xd0\x00\x02\x83\xd9\xfb\x9f\xdc\x1f\xfc\x20\x1e\x80\x00\x22\xc8\xf0\x00\x00\x66\x67\xa0\x00\x02\x00\x3d\x00\x00\x0f\xff\xe8\x00\x00\x7f\xee\xff\xdf\x40\x00\x0f\xf2\x78\x00\x01\x7f\x83\x9f\xf7\x9f\xfb\xc0\x00\x00\xff\xfe\x00\x00\x08\x00") + }, + { + DDCompatibilityTestSequence(), + BIN_STR("\x94\x74\x00\x00\x00\x9c\x00\x00\x00\x04\x00\x27\x00\x00\x00\x2a\x00\x00\x00\x00\x00\x00\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xbc\xe3\x5d\xa3\xd3\xd9\xf6\x1f\xe2\x07\x7c\x47\x20\x67\x48\x07\x47\xff\x47\xf6\xfe\xf8\x00\x00\x70\x6b\xd0\x00\x02\x83\xd9\xfb\x9f\xdc\x1f\xfc\x20\x1e\x80\x00\x22\xc8\xf0\x00\x00\x66\x67\xa0\x00\x02\x00\x3d\x00\x00\x0f\xff\xe8\x00\x00\x7f\xee\xff\xdf\x00\x00\x70\x0d\x7a\x00\x02\x80\x7b\x9f\xf7\x9f\xfb\xc0\x00\x00\xff\xfe\x00\x00\x08\x00") + }, + { + DDCompatibilityTestSequence(), + BIN_STR("\x94\xb5\x00\x00\x00\xcc\x00\x00\x00\x04\x00\x33\x00\x00\x00\x2a\x00\x00\x00\x00\x00\x00\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xbc\xe3\x5d\xa3\xd3\xd9\xf6\x1f\xe2\x07\x7c\x47\x20\x67\x48\x07\x47\xff\x47\xf6\xfe\xf8\x00\x00\x70\x6b\xd0\x00\x02\x83\xd9\xfb\x9f\xdc\x1f\xfc\x20\x1e\x80\x00\x22\xc8\xf0\x00\x00\x66\x67\xa0\x00\x02\x00\x3d\x00\x00\x0f\xff\xe8\x00\x00\x7f\xee\xff\xdf\x00\x00\x70\x0d\x7a\x00\x02\x80\x7b\x9f\xf7\x9f\xfb\xc0\x00\x00\xff\xfe\x00\x00\x08\x00\xf3\xff\xf9\x41\xaf\xbf\xff\xd6\x0c\xfc\xff\xff\xff\xfb\xf0\x00\x00\x00\x07\xff\xff\xff\xef\xc0\x00\x00\x00\x3f\xff\xff\xff\xfb\xff\xff\xff\xfa\x69\x74\xf3\xff\xff\xff\xe7\x9f\xff\xff\xff\x7e\x00\x00\x00\x00\xff\xff\xff\xfd\xf8\x00\x00\x00\x07\xff\xff\xff\xf0") + }, + { + DDCompatibilityTestSequence(), + BIN_STR("\x94\xd4\x00\x00\x00\x98\x01\x00\x00\x08\x00\x33\x00\x00\x00\x2a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xbc\xe3\x5d\xa3\xd3\xd9\xf6\x1f\xe2\x07\x7c\x47\x20\x67\x48\x07\x47\xff\x47\xf6\xfe\xf8\x00\x00\x70\x6b\xd0\x00\x02\x83\xd9\xfb\x9f\xdc\x1f\xfc\x20\x1e\x80\x00\x22\xc8\xf0\x00\x00\x66\x67\xa0\x00\x02\x00\x3d\x00\x00\x0f\xff\xe8\x00\x00\x7f\xee\xff\xdf\x00\x00\x70\x0d\x7a\x00\x02\x80\x7b\x9f\xf7\x9f\xfb\xc0\x00\x00\xff\xfe\x00\x00\x08\x00\xfc\x00\x00\x00\x04\x00\x06\xbe\x4f\xbf\xff\xd6\x0c\xff\x00\x00\x00\x01\x00\x00\x00\x03\xf8\x00\x00\x00\x08\x00\x00\x00\x0f\xc0\x00\x00\x00\x3f\xff\xff\xff\xfb\xff\xff\xff\xfb\xe0\x00\x00\x01\xc0\x00\x00\x06\x9f\x80\x00\x00\x0a\x00\x00\x00\x34\xf3\xff\xff\xff\xe7\x9f\xff\xff\xff\x7e\x00\x00\x00\x00\xff\xff\xff\xfd\xf0\x00\x00\x00\x07\xff\xff\xff\xf0") + }, + { + DDCompatibilityTestSequence(), + BIN_STR("\x94\xd4\x00\x00\x00\x98\x01\x00\x00\x08\x00\x33\x00\x00\x00\x2a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xbc\xe3\x5d\xa3\xd3\xd9\xf6\x1f\xe2\x07\x7c\x47\x20\x67\x48\x07\x47\xff\x47\xf6\xfe\xf8\x00\x00\x70\x6b\xd0\x00\x02\x83\xd9\xfb\x9f\xdc\x1f\xfc\x20\x1e\x80\x00\x22\xc8\xf0\x00\x00\x66\x67\xa0\x00\x02\x00\x3d\x00\x00\x0f\xff\xe8\x00\x00\x7f\xee\xff\xdf\x00\x00\x70\x0d\x7a\x00\x02\x80\x7b\x9f\xf7\x9f\xfb\xc0\x00\x00\xff\xfe\x00\x00\x08\x00\xfc\x00\x00\x00\x04\x00\x06\xbe\x4f\xbf\xff\xd6\x0c\xff\x00\x00\x00\x01\x00\x00\x00\x03\xf8\x00\x00\x00\x08\x00\x00\x00\x0f\xc0\x00\x00\x00\x3f\xff\xff\xff\xfb\xff\xff\xff\xfb\xe0\x00\x00\x01\xc0\x00\x00\x06\x9f\x80\x00\x00\x0a\x00\x00\x00\x34\xf3\xff\xff\xff\xe7\x9f\xff\xff\xff\x7e\x00\x00\x00\x00\xff\xff\xff\xfd\xf0\x00\x00\x00\x07\xff\xff\xff\xf0") + }, + }) + ) +); + +template +auto DDperformanceTestSequence() +{ + const auto times = 100'000; + return DDCompatibilityTestSequence() * times // average case + + generateSeq(G(MinMaxGenerator()), 0, times) // worst + + generateSeq(G(SameValueGenerator(42)), 0, times); // best +} + +// prime numbers in ascending order with some random repitions hit all the cases of Gorilla. +auto PrimesWithMultiplierGenerator = [](int multiplier = 1) +{ + return [multiplier](auto i) + { + static const int vals[] = { + 2, 3, 5, 7, 11, 11, 13, 17, 19, 23, 29, 29, 31, 37, 41, 43, + 47, 47, 53, 59, 61, 61, 67, 71, 73, 79, 83, 89, 89, 97, 101, 103, + 107, 107, 109, 113, 113, 127, 127, 127 + }; + static const size_t count = sizeof(vals)/sizeof(vals[0]); + + using T = decltype(i); + return static_cast(vals[i % count] * static_cast(multiplier)); + }; +}; + +template +auto GCompatibilityTestSequence() +{ + // Also multiply result by some factor to test large values on types that can hold those. + return generateSeq(G(PrimesWithMultiplierGenerator(intExp10(sizeof(ValueType)))), 0, 42); +} + +INSTANTIATE_TEST_SUITE_P(Gorilla, + CodecTestCompatibility, + ::testing::Combine( + ::testing::Values(Codec("Gorilla")), + ::testing::ValuesIn(std::initializer_list>{ + { + GCompatibilityTestSequence(), + BIN_STR("\x95\x35\x00\x00\x00\x2a\x00\x00\x00\x01\x00\x2a\x00\x00\x00\x14\xe1\xdd\x25\xe5\x7b\x29\x86\xee\x2a\x16\x5a\xc5\x0b\x23\x75\x1b\x3c\xb1\x97\x8b\x5f\xcb\x43\xd9\xc5\x48\xab\x23\xaf\x62\x93\x71\x4a\x73\x0f\xc6\x0a") + }, + { + GCompatibilityTestSequence(), + BIN_STR("\x95\x35\x00\x00\x00\x2a\x00\x00\x00\x01\x00\x2a\x00\x00\x00\x14\xe1\xdd\x25\xe5\x7b\x29\x86\xee\x2a\x16\x5a\xc5\x0b\x23\x75\x1b\x3c\xb1\x97\x8b\x5f\xcb\x43\xd9\xc5\x48\xab\x23\xaf\x62\x93\x71\x4a\x73\x0f\xc6\x0a") + }, + { + GCompatibilityTestSequence(), + BIN_STR("\x95\x52\x00\x00\x00\x54\x00\x00\x00\x02\x00\x2a\x00\x00\x00\xc8\x00\xdc\xfe\x66\xdb\x1f\x4e\xa7\xde\xdc\xd5\xec\x6e\xf7\x37\x3a\x23\xe7\x63\xf5\x6a\x8e\x99\x37\x34\xf9\xf8\x2e\x76\x35\x2d\x51\xbb\x3b\xc3\x6d\x13\xbf\x86\x53\x9e\x25\xe4\xaf\xaf\x63\xd5\x6a\x6e\x76\x35\x3a\x27\xd3\x0f\x91\xae\x6b\x33\x57\x6e\x64\xcc\x55\x81\xe4") + }, + { + GCompatibilityTestSequence(), + BIN_STR("\x95\x52\x00\x00\x00\x54\x00\x00\x00\x02\x00\x2a\x00\x00\x00\xc8\x00\xdc\xfe\x66\xdb\x1f\x4e\xa7\xde\xdc\xd5\xec\x6e\xf7\x37\x3a\x23\xe7\x63\xf5\x6a\x8e\x99\x37\x34\xf9\xf8\x2e\x76\x35\x2d\x51\xbb\x3b\xc3\x6d\x13\xbf\x86\x53\x9e\x25\xe4\xaf\xaf\x63\xd5\x6a\x6e\x76\x35\x3a\x27\xd3\x0f\x91\xae\x6b\x33\x57\x6e\x64\xcc\x55\x81\xe4") + }, + { + GCompatibilityTestSequence(), + BIN_STR("\x95\x65\x00\x00\x00\xa8\x00\x00\x00\x04\x00\x2a\x00\x00\x00\x20\x4e\x00\x00\xe4\x57\x63\xc0\xbb\x67\xbc\xce\x91\x97\x99\x15\x9e\xe3\x36\x3f\x89\x5f\x8e\xf2\xec\x8e\xd3\xbf\x75\x43\x58\xc4\x7e\xcf\x93\x43\x38\xc6\x91\x36\x1f\xe7\xb6\x11\x6f\x02\x73\x46\xef\xe0\xec\x50\xfb\x79\xcb\x9c\x14\xfa\x13\xea\x8d\x66\x43\x48\xa0\xde\x3a\xcf\xff\x26\xe0\x5f\x93\xde\x5e\x7f\x6e\x36\x5e\xe6\xb4\x66\x5d\xb0\x0e\xc4") + }, + { + GCompatibilityTestSequence(), + BIN_STR("\x95\x65\x00\x00\x00\xa8\x00\x00\x00\x04\x00\x2a\x00\x00\x00\x20\x4e\x00\x00\xe4\x57\x63\xc0\xbb\x67\xbc\xce\x91\x97\x99\x15\x9e\xe3\x36\x3f\x89\x5f\x8e\xf2\xec\x8e\xd3\xbf\x75\x43\x58\xc4\x7e\xcf\x93\x43\x38\xc6\x91\x36\x1f\xe7\xb6\x11\x6f\x02\x73\x46\xef\xe0\xec\x50\xfb\x79\xcb\x9c\x14\xfa\x13\xea\x8d\x66\x43\x48\xa0\xde\x3a\xcf\xff\x26\xe0\x5f\x93\xde\x5e\x7f\x6e\x36\x5e\xe6\xb4\x66\x5d\xb0\x0e\xc4") + }, + { + GCompatibilityTestSequence(), + BIN_STR("\x95\x91\x00\x00\x00\x50\x01\x00\x00\x08\x00\x2a\x00\x00\x00\x00\xc2\xeb\x0b\x00\x00\x00\x00\xe3\x2b\xa0\xa6\x19\x85\x98\xdc\x45\x74\x74\x43\xc2\x57\x41\x4c\x6e\x42\x79\xd9\x8f\x88\xa5\x05\xf3\xf1\x94\xa3\x62\x1e\x02\xdf\x05\x10\xf1\x15\x97\x35\x2a\x50\x71\x0f\x09\x6c\x89\xf7\x65\x1d\x11\xb7\xcc\x7d\x0b\x70\xc1\x86\x88\x48\x47\x87\xb6\x32\x26\xa7\x86\x87\x88\xd3\x93\x3d\xfc\x28\x68\x85\x05\x0b\x13\xc6\x5f\xd4\x70\xe1\x5e\x76\xf1\x9f\xf3\x33\x2a\x14\x14\x5e\x40\xc1\x5c\x28\x3f\xec\x43\x03\x05\x11\x91\xe8\xeb\x8e\x0a\x0e\x27\x21\x55\xcb\x39\xbc\x6a\xff\x11\x5d\x81\xa0\xa6\x10") + }, + { + GCompatibilityTestSequence(), + BIN_STR("\x95\x91\x00\x00\x00\x50\x01\x00\x00\x08\x00\x2a\x00\x00\x00\x00\xc2\xeb\x0b\x00\x00\x00\x00\xe3\x2b\xa0\xa6\x19\x85\x98\xdc\x45\x74\x74\x43\xc2\x57\x41\x4c\x6e\x42\x79\xd9\x8f\x88\xa5\x05\xf3\xf1\x94\xa3\x62\x1e\x02\xdf\x05\x10\xf1\x15\x97\x35\x2a\x50\x71\x0f\x09\x6c\x89\xf7\x65\x1d\x11\xb7\xcc\x7d\x0b\x70\xc1\x86\x88\x48\x47\x87\xb6\x32\x26\xa7\x86\x87\x88\xd3\x93\x3d\xfc\x28\x68\x85\x05\x0b\x13\xc6\x5f\xd4\x70\xe1\x5e\x76\xf1\x9f\xf3\x33\x2a\x14\x14\x5e\x40\xc1\x5c\x28\x3f\xec\x43\x03\x05\x11\x91\xe8\xeb\x8e\x0a\x0e\x27\x21\x55\xcb\x39\xbc\x6a\xff\x11\x5d\x81\xa0\xa6\x10") + }, + }) + ) +); + +// These 'tests' try to measure performance of encoding and decoding and hence only make sence to be run locally, +// also they require pretty big data to run agains and generating this data slows down startup of unit test process. +// So un-comment only at your discretion. + +// Just as if all sequences from generatePyramidOfSequences were appended to one-by-one to the first one. +//template +//CodecTestSequence generatePyramidSequence(const size_t sequences_count, Generator && generator, const char* generator_name) +//{ +// CodecTestSequence sequence; +// sequence.data_type = makeDataType(); +// sequence.serialized_data.reserve(sequences_count * sequences_count * sizeof(T)); +// +// for (size_t i = 1; i < sequences_count; ++i) +// { +// std::string name = generator_name + std::string(" from 0 to ") + std::to_string(i); +// sequence.append(generateSeq(std::forward(generator), name.c_str(), 0, i)); +// } +// +// return sequence; +//}; + +//INSTANTIATE_TEST_SUITE_P(DoubleDelta, +// CodecTestPerformance, +// ::testing::Combine( +// ::testing::Values(Codec("DoubleDelta")), +// ::testing::Values( +// DDperformanceTestSequence(), +// DDperformanceTestSequence(), +// DDperformanceTestSequence(), +// DDperformanceTestSequence(), +// DDperformanceTestSequence(), +// DDperformanceTestSequence(), +// DDperformanceTestSequence(), +// DDperformanceTestSequence() +// ) +// ), +//); + +//INSTANTIATE_TEST_SUITE_P(Gorilla, +// CodecTestPerformance, +// ::testing::Combine( +// ::testing::Values(Codec("Gorilla")), +// ::testing::Values( +// generatePyramidSequence(42, G(PrimesWithMultiplierGenerator())) * 6'000, +// generatePyramidSequence(42, G(PrimesWithMultiplierGenerator())) * 6'000, +// generatePyramidSequence(42, G(PrimesWithMultiplierGenerator())) * 6'000, +// generatePyramidSequence(42, G(PrimesWithMultiplierGenerator())) * 6'000, +// generatePyramidSequence(42, G(PrimesWithMultiplierGenerator())) * 6'000, +// generatePyramidSequence(42, G(PrimesWithMultiplierGenerator())) * 6'000, +// generatePyramidSequence(42, G(PrimesWithMultiplierGenerator())) * 6'000, +// generatePyramidSequence(42, G(PrimesWithMultiplierGenerator())) * 6'000 +// ) +// ), +//); + +} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 17d15a6643d..cd9de5abec3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -46,7 +46,7 @@ struct Settings : public SettingsCollection * A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions. */ -#define LIST_OF_SETTINGS(M) \ +#define COMMON_SETTINGS(M) \ M(SettingUInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \ M(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \ M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \ @@ -185,40 +185,10 @@ struct Settings : public SettingsCollection \ M(SettingString, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)", 0) \ \ - M(SettingBool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ - M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ - \ M(SettingBool, add_http_cors_header, false, "Write add http CORS header.", 0) \ \ M(SettingUInt64, max_http_get_redirects, 0, "Max number of http GET redirects hops allowed. Make sure additional security measures are in place to prevent a malicious server to redirect your requests to unexpected services.", 0) \ \ - M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \ - M(SettingBool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \ - M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ - M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \ - M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ - M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ - \ - M(SettingBool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \ - M(SettingBool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \ - M(SettingBool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ - M(SettingURI, format_avro_schema_registry_url, {}, "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ - \ - M(SettingBool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ - \ - M(SettingBool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \ - \ - M(SettingBool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \ - \ - M(SettingUInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \ - M(SettingUInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \ - M(SettingUInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \ - M(SettingBool, output_format_pretty_color, true, "Use ANSI escape sequences to paint colors in Pretty formats", 0) \ - M(SettingUInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \ - M(SettingString, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ - M(SettingUInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ - M(SettingBool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \ - \ M(SettingBool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", 0) \ \ M(SettingBool, send_progress_in_http_headers, false, "Send progress notifications using X-ClickHouse-Progress headers. Some clients do not support high amount of HTTP headers (Python requests in particular), so it is disabled by default.", 0) \ @@ -227,9 +197,6 @@ struct Settings : public SettingsCollection \ M(SettingBool, fsync_metadata, 1, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \ \ - M(SettingUInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ - M(SettingFloat, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ - \ M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \ \ M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ @@ -247,23 +214,6 @@ struct Settings : public SettingsCollection M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ - M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)", 0) \ - M(SettingString, format_template_resultset, "", "Path to file which contains format string for result set (for Template format)", 0) \ - M(SettingString, format_template_row, "", "Path to file which contains format string for rows (for Template format)", 0) \ - M(SettingString, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)", 0) \ - \ - M(SettingString, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_row_between_delimiter, "", "Delimiter between rows (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \ - \ - M(SettingString, format_regexp, "", "Regular expression (for Regexp format)", 0) \ - M(SettingString, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \ - M(SettingBool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ - \ M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ @@ -361,13 +311,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \ M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.", 0)\ M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.", 0) \ - M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ - M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \ - M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \ - M(SettingBool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \ - M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \ \ - M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \ M(SettingBool, log_query_settings, true, "Log query settings into the query_log.", 0) \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ @@ -446,7 +390,69 @@ struct Settings : public SettingsCollection M(SettingBool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \ M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ \ - M(SettingBool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \ + M(SettingBool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) + +#define FORMAT_FACTORY_SETTINGS(M) \ + M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ + M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \ + M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \ + M(SettingBool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \ + M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \ + M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \ + M(SettingBool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \ + M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ + M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \ + M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ + M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ + \ + M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ + \ + M(SettingBool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \ + M(SettingBool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \ + M(SettingBool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ + M(SettingURI, format_avro_schema_registry_url, {}, "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ + \ + M(SettingBool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ + \ + M(SettingBool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \ + \ + M(SettingBool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \ + \ + M(SettingUInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \ + M(SettingUInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \ + M(SettingUInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \ + M(SettingBool, output_format_pretty_color, true, "Use ANSI escape sequences to paint colors in Pretty formats", 0) \ + M(SettingUInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \ + M(SettingString, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ + M(SettingUInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ + M(SettingBool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \ + \ + M(SettingUInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ + M(SettingFloat, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ + \ + M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)", 0) \ + M(SettingString, format_template_resultset, "", "Path to file which contains format string for result set (for Template format)", 0) \ + M(SettingString, format_template_row, "", "Path to file which contains format string for rows (for Template format)", 0) \ + M(SettingString, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)", 0) \ + \ + M(SettingString, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_row_between_delimiter, "", "Delimiter between rows (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \ + \ + M(SettingString, format_regexp, "", "Regular expression (for Regexp format)", 0) \ + M(SettingString, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \ + M(SettingBool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ + \ + M(SettingBool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ + M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) + + #define LIST_OF_SETTINGS(M) \ + COMMON_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 878ab0c4e37..8e075e5bf08 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/src/DataStreams/tests/CMakeLists.txt b/src/DataStreams/tests/CMakeLists.txt index 95ef717c008..14db417b71c 100644 --- a/src/DataStreams/tests/CMakeLists.txt +++ b/src/DataStreams/tests/CMakeLists.txt @@ -1,13 +1,4 @@ set(SRCS) -add_executable (expression_stream expression_stream.cpp ${SRCS}) -target_link_libraries (expression_stream PRIVATE dbms clickhouse_storages_system clickhouse_parsers) - -add_executable (filter_stream filter_stream.cpp ${SRCS}) -target_link_libraries (filter_stream PRIVATE dbms clickhouse_storages_system clickhouse_parsers clickhouse_common_io) - -add_executable (union_stream2 union_stream2.cpp ${SRCS}) -target_link_libraries (union_stream2 PRIVATE dbms) - add_executable (finish_sorting_stream finish_sorting_stream.cpp ${SRCS}) target_link_libraries (finish_sorting_stream PRIVATE dbms) diff --git a/src/DataStreams/tests/expression_stream.cpp b/src/DataStreams/tests/expression_stream.cpp deleted file mode 100644 index 84b35cc2d3d..00000000000 --- a/src/DataStreams/tests/expression_stream.cpp +++ /dev/null @@ -1,86 +0,0 @@ -#include -#include - -#include -#include - -#include - -#include -#include -#include -#include - -#include - -#include -#include - -#include -#include -#include -#include -#include - - -int main(int argc, char ** argv) -try -{ - using namespace DB; - - size_t n = argc == 2 ? parse(argv[1]) : 10ULL; - - std::string input = "SELECT number, number / 3, number * number"; - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); - - SharedContextHolder shared_context = Context::createShared(); - Context context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); - - NamesAndTypesList source_columns = {{"number", std::make_shared()}}; - auto syntax_result = SyntaxAnalyzer(context).analyze(ast, source_columns); - SelectQueryExpressionAnalyzer analyzer(ast, syntax_result, context); - ExpressionActionsChain chain(context); - analyzer.appendSelect(chain, false); - analyzer.appendProjectResult(chain); - chain.finalize(); - ExpressionActionsPtr expression = chain.getLastActions(); - - StoragePtr table = StorageSystemNumbers::create(StorageID("test", "numbers"), false); - - Names column_names; - column_names.push_back("number"); - - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); - - BlockInputStreamPtr in; - in = std::make_shared(std::move(table->read(column_names, {}, context, stage, 8192, 1)[0])); - in = std::make_shared(in, expression); - in = std::make_shared(in, 10, std::max(static_cast(0), static_cast(n) - 10)); - - WriteBufferFromOStream out1(std::cout); - BlockOutputStreamPtr out = FormatFactory::instance().getOutput("TabSeparated", out1, expression->getSampleBlock(), context); - - { - Stopwatch stopwatch; - stopwatch.start(); - - copyData(*in, *out); - - stopwatch.stop(); - std::cout << std::fixed << std::setprecision(2) - << "Elapsed " << stopwatch.elapsedSeconds() << " sec." - << ", " << n / stopwatch.elapsedSeconds() << " rows/sec." - << std::endl; - } - - return 0; -} -catch (const DB::Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - throw; -} - diff --git a/src/DataStreams/tests/filter_stream.cpp b/src/DataStreams/tests/filter_stream.cpp deleted file mode 100644 index 8c481e1f258..00000000000 --- a/src/DataStreams/tests/filter_stream.cpp +++ /dev/null @@ -1,89 +0,0 @@ -#include -#include - -#include -#include - -#include - -#include -#include -#include -#include -#include - -#include - -#include -#include -#include - -#include -#include -#include -#include -#include - - -int main(int argc, char ** argv) -try -{ - using namespace DB; - - size_t n = argc == 2 ? parse(argv[1]) : 10ULL; - - std::string input = "SELECT number, number % 3 == 1"; - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); - - formatAST(*ast, std::cerr); - std::cerr << std::endl; - - SharedContextHolder shared_context = Context::createShared(); - Context context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); - - NamesAndTypesList source_columns = {{"number", std::make_shared()}}; - auto syntax_result = SyntaxAnalyzer(context).analyze(ast, source_columns); - SelectQueryExpressionAnalyzer analyzer(ast, syntax_result, context); - ExpressionActionsChain chain(context); - analyzer.appendSelect(chain, false); - analyzer.appendProjectResult(chain); - chain.finalize(); - ExpressionActionsPtr expression = chain.getLastActions(); - - StoragePtr table = StorageSystemNumbers::create(StorageID("test", "numbers"), false); - - Names column_names; - column_names.push_back("number"); - - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); - - BlockInputStreamPtr in = std::make_shared(std::move(table->read(column_names, {}, context, stage, 8192, 1)[0])); - in = std::make_shared(in, expression, "equals(modulo(number, 3), 1)"); - in = std::make_shared(in, 10, std::max(static_cast(0), static_cast(n) - 10)); - - WriteBufferFromOStream ob(std::cout); - BlockOutputStreamPtr out = FormatFactory::instance().getOutput("TabSeparated", ob, expression->getSampleBlock(), context); - - { - Stopwatch stopwatch; - stopwatch.start(); - - copyData(*in, *out); - - stopwatch.stop(); - std::cout << std::fixed << std::setprecision(2) - << "Elapsed " << stopwatch.elapsedSeconds() << " sec." - << ", " << n / stopwatch.elapsedSeconds() << " rows/sec." - << std::endl; - } - - return 0; -} -catch (const DB::Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - throw; -} diff --git a/src/DataStreams/tests/union_stream2.cpp b/src/DataStreams/tests/union_stream2.cpp deleted file mode 100644 index 5b84d89a435..00000000000 --- a/src/DataStreams/tests/union_stream2.cpp +++ /dev/null @@ -1,66 +0,0 @@ -#include -#include - -#include - -#include - -#include -#include -#include -#include -#include - -#include - -#include -#include -#include - - -using namespace DB; - -int main(int, char **) -try -{ - SharedContextHolder shared_context = Context::createShared(); - Context context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); - Settings settings = context.getSettings(); - - context.setPath("./"); - - loadMetadata(context); - - Names column_names; - column_names.push_back("WatchID"); - - StoragePtr table = DatabaseCatalog::instance().getTable({"default", "hits6"}, context); - - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); - auto pipes = table->read(column_names, {}, context, stage, settings.max_block_size, settings.max_threads); - - BlockInputStreams streams(pipes.size()); - - for (size_t i = 0, size = streams.size(); i < size; ++i) - streams[i] = std::make_shared(std::make_shared(std::move(pipes[i]))); - - BlockInputStreamPtr stream = std::make_shared(streams, nullptr, settings.max_threads); - stream = std::make_shared(stream, 10, 0); - - WriteBufferFromFileDescriptor wb(STDERR_FILENO); - Block sample = table->getSampleBlock(); - BlockOutputStreamPtr out = context.getOutputFormat("TabSeparated", wb, sample); - - copyData(*stream, *out); - - return 0; -} -catch (const Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl - << std::endl - << "Stack trace:" << std::endl - << e.getStackTraceString(); - return 1; -} diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index 814caeaffd3..aec6c779394 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -20,18 +20,42 @@ TEST(S3UriTest, validPatterns) ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("data", uri.key); + ASSERT_EQ(true, uri.is_virtual_hosted_style); } { S3::URI uri(Poco::URI("https://storage.yandexcloud.net/jokserfn/data")); ASSERT_EQ("https://storage.yandexcloud.net", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("data", uri.key); + ASSERT_EQ(false, uri.is_virtual_hosted_style); + } + { + S3::URI uri(Poco::URI("https://bucketname.s3.us-east-2.amazonaws.com/data")); + ASSERT_EQ("https://s3.us-east-2.amazonaws.com", uri.endpoint); + ASSERT_EQ("bucketname", uri.bucket); + ASSERT_EQ("data", uri.key); + ASSERT_EQ(true, uri.is_virtual_hosted_style); } { S3::URI uri(Poco::URI("https://s3.us-east-2.amazonaws.com/bucketname/data")); ASSERT_EQ("https://s3.us-east-2.amazonaws.com", uri.endpoint); ASSERT_EQ("bucketname", uri.bucket); ASSERT_EQ("data", uri.key); + ASSERT_EQ(false, uri.is_virtual_hosted_style); + } + { + S3::URI uri(Poco::URI("https://bucketname.s3-us-east-2.amazonaws.com/data")); + ASSERT_EQ("https://s3-us-east-2.amazonaws.com", uri.endpoint); + ASSERT_EQ("bucketname", uri.bucket); + ASSERT_EQ("data", uri.key); + ASSERT_EQ(true, uri.is_virtual_hosted_style); + } + { + S3::URI uri(Poco::URI("https://s3-us-east-2.amazonaws.com/bucketname/data")); + ASSERT_EQ("https://s3-us-east-2.amazonaws.com", uri.endpoint); + ASSERT_EQ("bucketname", uri.bucket); + ASSERT_EQ("data", uri.key); + ASSERT_EQ(false, uri.is_virtual_hosted_style); } } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 4f557111a3a..e7f6f16b91d 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -692,7 +692,8 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData if (auto * memory_tracker = memory_tracker_child->getParent()) current_memory_usage = memory_tracker->get(); - auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation; /// Here all the results in the sum are taken into account, from different threads. + /// Here all the results in the sum are taken into account, from different threads. + auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation; bool worth_convert_to_two_level = (params.group_by_two_level_threshold && result_size >= params.group_by_two_level_threshold) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 1c67991a20a..f9072e6176a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -664,8 +664,8 @@ void InterpreterSelectQuery::executeImpl(QueryPipeline & pipeline, const BlockIn { /** Streams of data. When the query is executed in parallel, we have several data streams. * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then - * if there is an ORDER BY, then glue the streams using UnionBlockInputStream, and then MergeSortingBlockInputStream, - * if not, then glue it using UnionBlockInputStream, + * if there is an ORDER BY, then glue the streams using ResizeProcessor, and then MergeSorting transforms, + * if not, then glue it using ResizeProcessor, * then apply LIMIT. * If there is GROUP BY, then we will perform all operations up to GROUP BY, inclusive, in parallel; * a parallel GROUP BY will glue streams into one, diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index eddfcd0a633..9ebdb155643 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -204,7 +204,6 @@ BlockIO InterpreterSystemQuery::execute() case Type::DROP_DNS_CACHE: context.checkAccess(AccessType::SYSTEM_DROP_DNS_CACHE); DNSResolver::instance().dropCache(); - AllowedClientHosts::dropDNSCaches(); /// Reinitialize clusters to update their resolved_addresses system_context.reloadClusterConfig(); break; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 354c7568818..04265734ce7 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #if defined(OS_LINUX) # include diff --git a/src/Interpreters/tests/CMakeLists.txt b/src/Interpreters/tests/CMakeLists.txt index 19d302d2b30..725af95563a 100644 --- a/src/Interpreters/tests/CMakeLists.txt +++ b/src/Interpreters/tests/CMakeLists.txt @@ -1,15 +1,3 @@ -add_executable (expression expression.cpp) -target_link_libraries (expression PRIVATE dbms clickhouse_parsers) - -add_executable (create_query create_query.cpp) -target_link_libraries (create_query PRIVATE dbms clickhouse_parsers) - -add_executable (select_query select_query.cpp) -target_link_libraries (select_query PRIVATE clickhouse_storages_system dbms clickhouse_common_io) - -add_executable (aggregate aggregate.cpp) -target_link_libraries (aggregate PRIVATE dbms) - add_executable (hash_map hash_map.cpp) target_include_directories (hash_map SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR}) target_link_libraries (hash_map PRIVATE dbms) diff --git a/src/Interpreters/tests/aggregate.cpp b/src/Interpreters/tests/aggregate.cpp deleted file mode 100644 index 9959bca7aac..00000000000 --- a/src/Interpreters/tests/aggregate.cpp +++ /dev/null @@ -1,105 +0,0 @@ -#include -#include - -#include -#include - -#include -#include - -#include - -#include - -#include - - -int main(int argc, char ** argv) -{ - using namespace DB; - - try - { - size_t n = argc == 2 ? std::stol(argv[1]) : 10; - - Block block; - - { - ColumnWithTypeAndName column; - column.name = "x"; - column.type = std::make_shared(); - auto col = ColumnInt16::create(); - auto & vec_x = col->getData(); - - vec_x.resize(n); - for (size_t i = 0; i < n; ++i) - vec_x[i] = i % 9; - - column.column = std::move(col); - block.insert(column); - } - - const char * strings[] = {"abc", "def", "abcd", "defg", "ac"}; - - { - ColumnWithTypeAndName column; - column.name = "s1"; - column.type = std::make_shared(); - auto col = ColumnString::create(); - - for (size_t i = 0; i < n; ++i) - col->insert(std::string(strings[i % 5])); - - column.column = std::move(col); - block.insert(column); - } - - { - ColumnWithTypeAndName column; - column.name = "s2"; - column.type = std::make_shared(); - auto col = ColumnString::create(); - - for (size_t i = 0; i < n; ++i) - col->insert(std::string(strings[i % 3])); - - column.column = std::move(col); - block.insert(column); - } - - BlockInputStreamPtr stream = std::make_shared(block); - AggregatedDataVariants aggregated_data_variants; - - AggregateFunctionFactory factory; - - AggregateDescriptions aggregate_descriptions(1); - - DataTypes empty_list_of_types; - aggregate_descriptions[0].function = factory.get("count", empty_list_of_types); - - Aggregator::Params params( - stream->getHeader(), {0, 1}, aggregate_descriptions, - false, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, 1, 0); - - Aggregator aggregator(params); - - { - Stopwatch stopwatch; - stopwatch.start(); - - aggregator.execute(stream, aggregated_data_variants); - - stopwatch.stop(); - std::cout << std::fixed << std::setprecision(2) - << "Elapsed " << stopwatch.elapsedSeconds() << " sec." - << ", " << n / stopwatch.elapsedSeconds() << " rows/sec." - << std::endl; - } - } - catch (const Exception & e) - { - std::cerr << e.displayText() << std::endl; - } - - return 0; -} diff --git a/src/Interpreters/tests/create_query.cpp b/src/Interpreters/tests/create_query.cpp deleted file mode 100644 index 82bb8db5d78..00000000000 --- a/src/Interpreters/tests/create_query.cpp +++ /dev/null @@ -1,103 +0,0 @@ -#include -#include - -#include -#include -#include - -#include - -#include -#include - - -using namespace DB; - -int main(int, char **) -try -{ - std::string input = "CREATE TABLE IF NOT EXISTS hits (\n" - "WatchID UInt64,\n" - "JavaEnable UInt8,\n" - "Title String,\n" - "EventTime DateTime,\n" - "CounterID UInt32,\n" - "ClientIP UInt32,\n" - "RegionID UInt32,\n" - "UniqID UInt64,\n" - "CounterClass UInt8,\n" - "OS UInt8,\n" - "UserAgent UInt8,\n" - "URL String,\n" - "Referer String,\n" - "ResolutionWidth UInt16,\n" - "ResolutionHeight UInt16,\n" - "ResolutionDepth UInt8,\n" - "FlashMajor UInt8,\n" - "FlashMinor UInt8,\n" - "FlashMinor2 String,\n" - "NetMajor UInt8,\n" - "NetMinor UInt8,\n" - "UserAgentMajor UInt16,\n" - "UserAgentMinor FixedString(2),\n" - "CookieEnable UInt8,\n" - "JavascriptEnable UInt8,\n" - "IsMobile UInt8,\n" - "MobilePhone UInt8,\n" - "MobilePhoneModel String,\n" - "Params String,\n" - "IPNetworkID UInt32,\n" - "TraficSourceID Int8,\n" - "SearchEngineID UInt16,\n" - "SearchPhrase String,\n" - "AdvEngineID UInt8,\n" - "IsArtifical UInt8,\n" - "WindowClientWidth UInt16,\n" - "WindowClientHeight UInt16,\n" - "ClientTimeZone Int16,\n" - "ClientEventTime DateTime,\n" - "SilverlightVersion1 UInt8,\n" - "SilverlightVersion2 UInt8,\n" - "SilverlightVersion3 UInt32,\n" - "SilverlightVersion4 UInt16,\n" - "PageCharset String,\n" - "CodeVersion UInt32,\n" - "IsLink UInt8,\n" - "IsDownload UInt8,\n" - "IsNotBounce UInt8,\n" - "FUniqID UInt64,\n" - "OriginalURL String,\n" - "HID UInt32,\n" - "IsOldCounter UInt8,\n" - "IsEvent UInt8,\n" - "IsParameter UInt8,\n" - "DontCountHits UInt8,\n" - "WithHash UInt8\n" - ") ENGINE = Log"; - - ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); - - SharedContextHolder shared_context = Context::createShared(); - Context context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); - - context.setPath("./"); - auto database = std::make_shared("test", "./metadata/test/", context); - DatabaseCatalog::instance().attachDatabase("test", database); - database->loadStoredObjects(context, false); - context.setCurrentDatabase("test"); - - InterpreterCreateQuery interpreter(ast, context); - interpreter.execute(); - - return 0; -} -catch (const Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl - << std::endl - << "Stack trace:" << std::endl - << e.getStackTraceString(); - return 1; -} diff --git a/src/Interpreters/tests/expression.cpp b/src/Interpreters/tests/expression.cpp deleted file mode 100644 index 8327514b3d3..00000000000 --- a/src/Interpreters/tests/expression.cpp +++ /dev/null @@ -1,140 +0,0 @@ -#include -#include - -#include - -#include -#include - -#include -#include - -#include -#include -#include -#include - -#include -#include -#include -#include - -#include -#include -#include -#include - - -int main(int argc, char ** argv) -{ - using namespace DB; - - try - { - std::string input = "SELECT x, s1, s2, " - "/*" - "2 + x * 2, x * 2, x % 3 == 1, " - "s1 == 'abc', s1 == s2, s1 != 'abc', s1 != s2, " - "s1 < 'abc', s1 < s2, s1 > 'abc', s1 > s2, " - "s1 <= 'abc', s1 <= s2, s1 >= 'abc', s1 >= s2, " - "*/" - "s1 < s2 AND x % 3 < x % 5"; - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); - - formatAST(*ast, std::cerr); - std::cerr << std::endl; - - SharedContextHolder shared_context = Context::createShared(); - Context context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); - NamesAndTypesList columns - { - {"x", std::make_shared()}, - {"s1", std::make_shared()}, - {"s2", std::make_shared()} - }; - - auto syntax_result = SyntaxAnalyzer(context).analyze(ast, columns); - SelectQueryExpressionAnalyzer analyzer(ast, syntax_result, context); - ExpressionActionsChain chain(context); - analyzer.appendSelect(chain, false); - analyzer.appendProjectResult(chain); - chain.finalize(); - ExpressionActionsPtr expression = chain.getLastActions(); - - size_t n = argc == 2 ? std::stol(argv[1]) : 10; - - Block block; - - { - ColumnWithTypeAndName column; - column.name = "x"; - column.type = std::make_shared(); - auto col = ColumnInt16::create(); - auto & vec_x = col->getData(); - - vec_x.resize(n); - for (size_t i = 0; i < n; ++i) - vec_x[i] = i % 9; - - column.column = std::move(col); - block.insert(column); - } - - const char * strings[] = {"abc", "def", "abcd", "defg", "ac"}; - - { - ColumnWithTypeAndName column; - column.name = "s1"; - column.type = std::make_shared(); - auto col = ColumnString::create(); - - for (size_t i = 0; i < n; ++i) - col->insert(std::string(strings[i % 5])); - - column.column = std::move(col); - block.insert(column); - } - - { - ColumnWithTypeAndName column; - column.name = "s2"; - column.type = std::make_shared(); - auto col = ColumnString::create(); - - for (size_t i = 0; i < n; ++i) - col->insert(std::string(strings[i % 3])); - - column.column = std::move(col); - block.insert(column); - } - - { - Stopwatch stopwatch; - stopwatch.start(); - - expression->execute(block); - - stopwatch.stop(); - std::cout << std::fixed << std::setprecision(2) - << "Elapsed " << stopwatch.elapsedSeconds() << " sec." - << ", " << n / stopwatch.elapsedSeconds() << " rows/sec." - << std::endl; - } - - auto is = std::make_shared(block); - LimitBlockInputStream lis(is, 20, std::max(0, static_cast(n) - 20)); - WriteBufferFromOStream out_buf(std::cout); - BlockOutputStreamPtr out = FormatFactory::instance().getOutput("TabSeparated", out_buf, block, context); - - copyData(lis, *out); - } - catch (const Exception & e) - { - std::cerr << e.displayText() << std::endl; - } - - return 0; -} diff --git a/src/Interpreters/tests/select_query.cpp b/src/Interpreters/tests/select_query.cpp deleted file mode 100644 index fb364d28086..00000000000 --- a/src/Interpreters/tests/select_query.cpp +++ /dev/null @@ -1,61 +0,0 @@ -#include -#include - -#include - -#include - -#include -#include - -#include -#include - -#include -#include -#include -#include -#include - - -using namespace DB; - -int main(int, char **) -try -{ - Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Poco::Logger::root().setChannel(channel); - Poco::Logger::root().setLevel("trace"); - - /// Pre-initialize the `DateLUT` so that the first initialization does not affect the measured execution speed. - DateLUT::instance(); - - SharedContextHolder shared_context = Context::createShared(); - Context context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); - - context.setPath("./"); - - loadMetadata(context); - - DatabasePtr system = std::make_shared("system", "./metadata/system/", context); - DatabaseCatalog::instance().attachDatabase("system", system); - system->loadStoredObjects(context, false); - attachSystemTablesLocal(*DatabaseCatalog::instance().getSystemDatabase()); - context.setCurrentDatabase("default"); - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - executeQuery(in, out, /* allow_into_outfile = */ false, context, {}); - - return 0; -} -catch (const Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl - << std::endl - << "Stack trace:" << std::endl - << e.getStackTraceString(); - return 1; -} diff --git a/src/Processors/tests/CMakeLists.txt b/src/Processors/tests/CMakeLists.txt index 4ddb6c68416..e69de29bb2d 100644 --- a/src/Processors/tests/CMakeLists.txt +++ b/src/Processors/tests/CMakeLists.txt @@ -1,15 +0,0 @@ -add_executable (processors_test processors_test.cpp) -add_executable (processors_test_chain processors_test_chain.cpp) -add_executable (processors_test_merge processors_test_merge.cpp) -add_executable (processors_test_merging_sorted_transform processors_test_merging_sorted_transform.cpp) -add_executable (processors_test_merge_sorting_transform processors_test_merge_sorting_transform.cpp) -add_executable (processors_test_expand_pipeline processors_test_expand_pipeline.cpp) -add_executable (processors_test_aggregation processors_test_aggregation.cpp) - -target_link_libraries (processors_test PRIVATE dbms) -target_link_libraries (processors_test_chain PRIVATE dbms) -target_link_libraries (processors_test_merge PRIVATE dbms) -target_link_libraries (processors_test_expand_pipeline PRIVATE dbms) -target_link_libraries (processors_test_merging_sorted_transform PRIVATE dbms) -target_link_libraries (processors_test_merge_sorting_transform PRIVATE dbms) -target_link_libraries (processors_test_aggregation PRIVATE dbms clickhouse_aggregate_functions) diff --git a/src/Processors/tests/processors_test.cpp b/src/Processors/tests/processors_test.cpp deleted file mode 100644 index 3c73223e59c..00000000000 --- a/src/Processors/tests/processors_test.cpp +++ /dev/null @@ -1,228 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include - - -using namespace DB; - - -class NumbersSource : public ISource -{ -public: - String getName() const override { return "Numbers"; } - - NumbersSource(UInt64 start_number, unsigned sleep_useconds_) - : ISource(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - current_number(start_number), sleep_useconds(sleep_useconds_) - { - } - -private: - UInt64 current_number = 0; - unsigned sleep_useconds; - - Chunk generate() override - { - usleep(sleep_useconds); - - MutableColumns columns; - columns.emplace_back(ColumnUInt64::create(1, current_number)); - ++current_number; - return Chunk(std::move(columns), 1); - } -}; - - -class SleepyNumbersSource : public IProcessor -{ -protected: - OutputPort & output; - -public: - String getName() const override { return "SleepyNumbers"; } - - SleepyNumbersSource(UInt64 start_number, unsigned sleep_useconds_) - : IProcessor({}, {Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})}) - , output(outputs.front()), current_number(start_number), sleep_useconds(sleep_useconds_) - { - } - - Status prepare() override - { - if (active) - return Status::Wait; - - if (output.isFinished()) - return Status::Finished; - - if (!output.canPush()) - return Status::PortFull; - - if (!current_chunk) - return Status::Async; - - output.push(std::move(current_chunk)); - return Status::Async; - } - - void schedule(EventCounter & watch) override - { - active = true; - pool.scheduleOrThrowOnError([&watch, this] - { - usleep(sleep_useconds); - current_chunk = generate(); - active = false; - watch.notify(); - }); - } - - OutputPort & getPort() { return output; } - -private: - ThreadPool pool{1, 1, 0}; - Chunk current_chunk; - std::atomic_bool active {false}; - - UInt64 current_number = 0; - unsigned sleep_useconds; - - Chunk generate() - { - MutableColumns columns; - columns.emplace_back(ColumnUInt64::create(1, current_number)); - ++current_number; - return Chunk(std::move(columns), 1); - } -}; - - -class PrintSink : public ISink -{ -public: - String getName() const override { return "Print"; } - - explicit PrintSink(String prefix_) - : ISink(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - prefix(std::move(prefix_)) - { - } - -private: - String prefix; - WriteBufferFromFileDescriptor out{STDOUT_FILENO}; - FormatSettings settings; - - void consume(Chunk chunk) override - { - size_t rows = chunk.getNumRows(); - size_t columns = chunk.getNumColumns(); - - for (size_t row_num = 0; row_num < rows; ++row_num) - { - writeString(prefix, out); - for (size_t column_num = 0; column_num < columns; ++column_num) - { - if (column_num != 0) - writeChar('\t', out); - getPort().getHeader().getByPosition(column_num).type->serializeAsText(*chunk.getColumns()[column_num], row_num, out, settings); - } - writeChar('\n', out); - } - - out.next(); - } -}; - - -int main(int, char **) -try -{ - auto source0 = std::make_shared(0, 300000); - auto header = source0->getPort().getHeader(); - auto limit0 = std::make_shared(header, 10, 0); - - connect(source0->getPort(), limit0->getInputPort()); - - auto queue = std::make_shared(header); - - connect(limit0->getOutputPort(), queue->getInputPort()); - - auto source1 = std::make_shared(100, 100000); - auto source2 = std::make_shared(1000, 200000); - - auto source3 = std::make_shared(10, 100000); - auto limit3 = std::make_shared(header, 5, 0); - - connect(source3->getPort(), limit3->getInputPort()); - - auto source4 = std::make_shared(10, 100000); - auto limit4 = std::make_shared(header, 5, 0); - - connect(source4->getPort(), limit4->getInputPort()); - - auto concat = std::make_shared(header, 2); - - connect(limit3->getOutputPort(), concat->getInputs().front()); - connect(limit4->getOutputPort(), concat->getInputs().back()); - - auto fork = std::make_shared(header, 2); - - connect(concat->getOutputPort(), fork->getInputPort()); - - auto print_after_concat = std::make_shared("---------- "); - - connect(fork->getOutputs().back(), print_after_concat->getPort()); - - auto resize = std::make_shared(header, 4, 1); - - auto input_it = resize->getInputs().begin(); - connect(queue->getOutputPort(), *(input_it++)); - connect(source1->getPort(), *(input_it++)); - connect(source2->getPort(), *(input_it++)); - connect(fork->getOutputs().front(), *(input_it++)); - - auto limit = std::make_shared(header, 100, 0); - - connect(resize->getOutputs().front(), limit->getInputPort()); - - auto sink = std::make_shared(""); - - connect(limit->getOutputPort(), sink->getPort()); - - WriteBufferFromOStream out(std::cout); - std::vector processors = {source0, source1, source2, source3, source4, limit0, limit3, limit4, limit, - queue, concat, fork, print_after_concat, resize, sink}; - printPipeline(processors, out); - - // ThreadPool pool(4, 4, 10); - PipelineExecutor executor(processors); - /// SequentialPipelineExecutor executor({sink}); - - executor.execute(1); - - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; -} diff --git a/src/Processors/tests/processors_test_aggregation.cpp b/src/Processors/tests/processors_test_aggregation.cpp deleted file mode 100644 index 9b8bee67d52..00000000000 --- a/src/Processors/tests/processors_test_aggregation.cpp +++ /dev/null @@ -1,411 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -using namespace DB; - -namespace DB::ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -class NumbersSource : public ISource -{ -public: - String getName() const override { return "Numbers"; } - - NumbersSource(UInt64 start_number, UInt64 step_, UInt64 block_size_, unsigned sleep_useconds_) - : ISource(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - current_number(start_number), step(step_), block_size(block_size_), sleep_useconds(sleep_useconds_) - { - } - -private: - UInt64 current_number = 0; - UInt64 step; - UInt64 block_size; - unsigned sleep_useconds; - - Chunk generate() override - { - usleep(sleep_useconds); - - MutableColumns columns; - columns.emplace_back(ColumnUInt64::create()); - - for (UInt64 i = 0; i < block_size; ++i, current_number += step) - columns.back()->insert(Field(current_number)); - - return Chunk(std::move(columns), block_size); - } -}; - -class PrintSink : public ISink -{ -public: - String getName() const override { return "Print"; } - - PrintSink(String prefix_, Block header) - : ISink(std::move(header)), - prefix(std::move(prefix_)) - { - } - -private: - String prefix; - WriteBufferFromFileDescriptor out{STDOUT_FILENO}; - FormatSettings settings; - - void consume(Chunk chunk) override - { - size_t rows = chunk.getNumRows(); - size_t columns = chunk.getNumColumns(); - - for (size_t row_num = 0; row_num < rows; ++row_num) - { - writeString(prefix, out); - for (size_t column_num = 0; column_num < columns; ++column_num) - { - if (column_num != 0) - writeChar('\t', out); - getPort().getHeader().getByPosition(column_num).type->serializeAsText(*chunk.getColumns()[column_num], row_num, out, settings); - } - writeChar('\n', out); - } - - out.next(); - } -}; - -class CheckSink : public ISink -{ -public: - String getName() const override { return "Check"; } - - CheckSink(Block header, size_t num_rows) - : ISink(std::move(header)), read_rows(num_rows, false) - { - } - - void checkAllRead() - { - for (size_t i = 0; i < read_rows.size(); ++i) - { - if (!read_rows[i]) - { - throw Exception("Check Failed. Row " + toString(i) + " was not read.", ErrorCodes::LOGICAL_ERROR); - } - } - } - -private: - std::vector read_rows; - - void consume(Chunk chunk) override - { - size_t rows = chunk.getNumRows(); - size_t columns = chunk.getNumColumns(); - - for (size_t row_num = 0; row_num < rows; ++row_num) - { - std::vector values(columns); - for (size_t column_num = 0; column_num < columns; ++column_num) - { - values[column_num] = chunk.getColumns()[column_num]->getUInt(row_num); - } - - if (values.size() >= 2 && 3 * values[0] != values[1]) - throw Exception("Check Failed. Got (" + toString(values[0]) + ", " + toString(values[1]) + ") in result," - + "but " + toString(values[0]) + " * 3 != " + toString(values[1]), - ErrorCodes::LOGICAL_ERROR); - - if (values[0] >= read_rows.size()) - throw Exception("Check Failed. Got string with number " + toString(values[0]) + - " (max " + toString(read_rows.size()), ErrorCodes::LOGICAL_ERROR); - - if (read_rows[values[0]]) - throw Exception("Row " + toString(values[0]) + " was already read.", ErrorCodes::LOGICAL_ERROR); - - read_rows[values[0]] = true; - } - } -}; - -template -struct Measure -{ - template - static typename TimeT::rep execution(F&& func, Args&&... args) - { - auto start = std::chrono::steady_clock::now(); - std::forward(func)(std::forward(args)...); - auto duration = std::chrono::duration_cast< TimeT> - (std::chrono::steady_clock::now() - start); - return duration.count(); - } -}; - -int main(int, char **) -try -{ - ThreadStatus thread_status; - CurrentThread::initializeQuery(); - auto thread_group = CurrentThread::getGroup(); - - Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Poco::Logger::root().setChannel(channel); - Poco::Logger::root().setLevel("trace"); - - registerAggregateFunctions(); - auto & factory = AggregateFunctionFactory::instance(); - - auto cur_path = Poco::Path().absolute().toString(); - auto disk = std::make_shared("tmp", cur_path, 0); - auto tmp_volume = std::make_shared("tmp", std::vector{disk}, 0); - - auto execute_one_stream = [&](String msg, size_t num_threads, bool two_level, bool external) - { - std::cerr << '\n' << msg << "\n"; - - size_t num_rows = 1000000; - size_t block_size = 1000; - - auto source1 = std::make_shared(0, 1, block_size, 0); - auto source2 = std::make_shared(0, 1, block_size, 0); - auto source3 = std::make_shared(0, 1, block_size, 0); - - auto limit1 = std::make_shared(source1->getPort().getHeader(), num_rows, 0); - auto limit2 = std::make_shared(source2->getPort().getHeader(), num_rows, 0); - auto limit3 = std::make_shared(source3->getPort().getHeader(), num_rows, 0); - - auto resize = std::make_shared(source1->getPort().getHeader(), 3, 1); - - AggregateDescriptions aggregate_descriptions(1); - - DataTypes sum_types = { std::make_shared() }; - aggregate_descriptions[0].function = factory.get("sum", sum_types); - aggregate_descriptions[0].arguments = {0}; - - bool overflow_row = false; /// Without overflow row. - size_t max_rows_to_group_by = 0; /// All. - size_t group_by_two_level_threshold = two_level ? 10 : 0; - size_t group_by_two_level_threshold_bytes = two_level ? 128 : 0; - size_t max_bytes_before_external_group_by = external ? 10000000 : 0; - - Aggregator::Params params( - source1->getPort().getHeader(), - {0}, - aggregate_descriptions, - overflow_row, - max_rows_to_group_by, - OverflowMode::THROW, - group_by_two_level_threshold, - group_by_two_level_threshold_bytes, - max_bytes_before_external_group_by, - false, /// empty_result_for_aggregation_by_empty_set - tmp_volume, - 1, /// max_threads - 0 - ); - - auto agg_params = std::make_shared(params, /* final =*/ false); - auto merge_params = std::make_shared(params, /* final =*/ true); - auto aggregating = std::make_shared(source1->getPort().getHeader(), agg_params); - auto merging = std::make_shared(aggregating->getOutputs().front().getHeader(), merge_params, 4); - auto sink = std::make_shared(merging->getOutputPort().getHeader(), num_rows); - - connect(source1->getPort(), limit1->getInputPort()); - connect(source2->getPort(), limit2->getInputPort()); - connect(source3->getPort(), limit3->getInputPort()); - - auto it = resize->getInputs().begin(); - connect(limit1->getOutputPort(), *(it++)); - connect(limit2->getOutputPort(), *(it++)); - connect(limit3->getOutputPort(), *(it++)); - - connect(resize->getOutputs().front(), aggregating->getInputs().front()); - connect(aggregating->getOutputs().front(), merging->getInputPort()); - connect(merging->getOutputPort(), sink->getPort()); - - std::vector processors = {source1, source2, source3, - limit1, limit2, limit3, - resize, aggregating, merging, sink}; -// WriteBufferFromOStream out(std::cout); -// printPipeline(processors, out); - - PipelineExecutor executor(processors); - executor.execute(num_threads); - sink->checkAllRead(); - }; - - auto execute_mult_streams = [&](String msg, size_t num_threads, bool two_level, bool external) - { - std::cerr << '\n' << msg << "\n"; - - size_t num_rows = 1000000; - size_t block_size = 1000; - - auto source1 = std::make_shared(0, 1, block_size, 0); - auto source2 = std::make_shared(0, 1, block_size, 0); - auto source3 = std::make_shared(0, 1, block_size, 0); - - auto limit1 = std::make_shared(source1->getPort().getHeader(), num_rows, 0); - auto limit2 = std::make_shared(source2->getPort().getHeader(), num_rows, 0); - auto limit3 = std::make_shared(source3->getPort().getHeader(), num_rows, 0); - - AggregateDescriptions aggregate_descriptions(1); - - DataTypes sum_types = { std::make_shared() }; - aggregate_descriptions[0].function = factory.get("sum", sum_types); - aggregate_descriptions[0].arguments = {0}; - - bool overflow_row = false; /// Without overflow row. - size_t max_rows_to_group_by = 0; /// All. - size_t group_by_two_level_threshold = two_level ? 10 : 0; - size_t group_by_two_level_threshold_bytes = two_level ? 128 : 0; - size_t max_bytes_before_external_group_by = external ? 10000000 : 0; - - Aggregator::Params params( - source1->getPort().getHeader(), - {0}, - aggregate_descriptions, - overflow_row, - max_rows_to_group_by, - OverflowMode::THROW, - group_by_two_level_threshold, - group_by_two_level_threshold_bytes, - max_bytes_before_external_group_by, - false, /// empty_result_for_aggregation_by_empty_set - tmp_volume, - 1, /// max_threads - 0 - ); - - auto agg_params = std::make_shared(params, /* final =*/ false); - auto merge_params = std::make_shared(params, /* final =*/ true); - - ManyAggregatedDataPtr data = std::make_unique(3); - - auto aggregating1 = std::make_shared(source1->getPort().getHeader(), agg_params, data, 0, 4, 4); - auto aggregating2 = std::make_shared(source1->getPort().getHeader(), agg_params, data, 1, 4, 4); - auto aggregating3 = std::make_shared(source1->getPort().getHeader(), agg_params, data, 2, 4, 4); - - Processors merging_pipe = createMergingAggregatedMemoryEfficientPipe( - aggregating1->getOutputs().front().getHeader(), - merge_params, - 3, 2); - - auto sink = std::make_shared(merging_pipe.back()->getOutputs().back().getHeader(), num_rows); - - connect(source1->getPort(), limit1->getInputPort()); - connect(source2->getPort(), limit2->getInputPort()); - connect(source3->getPort(), limit3->getInputPort()); - - connect(limit1->getOutputPort(), aggregating1->getInputs().front()); - connect(limit2->getOutputPort(), aggregating2->getInputs().front()); - connect(limit3->getOutputPort(), aggregating3->getInputs().front()); - - auto it = merging_pipe.front()->getInputs().begin(); - connect(aggregating1->getOutputs().front(), *(it++)); - connect(aggregating2->getOutputs().front(), *(it++)); - connect(aggregating3->getOutputs().front(), *(it++)); - - connect(merging_pipe.back()->getOutputs().back(), sink->getPort()); - - std::vector processors = {source1, source2, source3, - limit1, limit2, limit3, - aggregating1, aggregating2, aggregating3, sink}; - - processors.insert(processors.end(), merging_pipe.begin(), merging_pipe.end()); -// WriteBufferFromOStream out(std::cout); -// printPipeline(processors, out); - - PipelineExecutor executor(processors); - executor.execute(num_threads); - sink->checkAllRead(); - }; - - std::vector messages; - std::vector times; - - auto exec = [&](auto func, String msg, size_t num_threads, bool two_level, bool external) - { - msg += ", two_level = " + toString(two_level) + ", external = " + toString(external); - Int64 time = 0; - - auto wrapper = [&]() - { - ThreadStatus cur_status; - - CurrentThread::attachToIfDetached(thread_group); - time = Measure<>::execution(func, msg, num_threads, two_level, external); - }; - - std::thread thread(wrapper); - thread.join(); - - messages.emplace_back(msg); - times.emplace_back(time); - }; - - size_t num_threads = 4; - - exec(execute_one_stream, "One stream, single thread", 1, false, false); - exec(execute_one_stream, "One stream, multiple threads", num_threads, false, false); - - exec(execute_mult_streams, "Multiple streams, single thread", 1, false, false); - exec(execute_mult_streams, "Multiple streams, multiple threads", num_threads, false, false); - - exec(execute_one_stream, "One stream, single thread", 1, true, false); - exec(execute_one_stream, "One stream, multiple threads", num_threads, true, false); - - exec(execute_mult_streams, "Multiple streams, single thread", 1, true, false); - exec(execute_mult_streams, "Multiple streams, multiple threads", num_threads, true, false); - - exec(execute_one_stream, "One stream, single thread", 1, true, true); - exec(execute_one_stream, "One stream, multiple threads", num_threads, true, true); - - exec(execute_mult_streams, "Multiple streams, single thread", 1, true, true); - exec(execute_mult_streams, "Multiple streams, multiple threads", num_threads, true, true); - - for (size_t i = 0; i < messages.size(); ++i) - std::cout << messages[i] << " time: " << times[i] << " ms.\n"; - - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; -} diff --git a/src/Processors/tests/processors_test_chain.cpp b/src/Processors/tests/processors_test_chain.cpp deleted file mode 100644 index 0fbd52eef39..00000000000 --- a/src/Processors/tests/processors_test_chain.cpp +++ /dev/null @@ -1,165 +0,0 @@ -#include - -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include - -#include -#include - - -using namespace DB; - - -class NumbersSource : public ISource -{ -public: - String getName() const override { return "Numbers"; } - - NumbersSource(UInt64 start_number, unsigned sleep_useconds_) - : ISource(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - current_number(start_number), sleep_useconds(sleep_useconds_) - { - } - -private: - UInt64 current_number = 0; - unsigned sleep_useconds; - - Chunk generate() override - { - usleep(sleep_useconds); - - MutableColumns columns; - columns.emplace_back(ColumnUInt64::create(1, current_number)); - ++current_number; - return Chunk(std::move(columns), 1); - } -}; - -class SleepyTransform : public ISimpleTransform -{ -public: - explicit SleepyTransform(unsigned sleep_useconds_) - : ISimpleTransform( - Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }}), - Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }}), - /*skip_empty_chunks =*/ false) - , sleep_useconds(sleep_useconds_) {} - - String getName() const override { return "SleepyTransform"; } - -protected: - void transform(Chunk &) override - { - usleep(sleep_useconds); - } - -private: - unsigned sleep_useconds; -}; - -class PrintSink : public ISink -{ -public: - String getName() const override { return "Print"; } - - explicit PrintSink(String prefix_) - : ISink(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - prefix(std::move(prefix_)) - { - } - -private: - String prefix; - WriteBufferFromFileDescriptor out{STDOUT_FILENO}; - FormatSettings settings; - - void consume(Chunk chunk) override - { - size_t rows = chunk.getNumRows(); - size_t columns = chunk.getNumColumns(); - - for (size_t row_num = 0; row_num < rows; ++row_num) - { - writeString(prefix, out); - for (size_t column_num = 0; column_num < columns; ++column_num) - { - if (column_num != 0) - writeChar('\t', out); - getPort().getHeader().getByPosition(column_num).type->serializeAsText(*chunk.getColumns()[column_num], row_num, out, settings); - } - writeChar('\n', out); - } - - out.next(); - } -}; - -template -struct Measure -{ - template - static typename TimeT::rep execution(F&& func, Args&&... args) - { - auto start = std::chrono::steady_clock::now(); - std::forward(func)(std::forward(args)...); - auto duration = std::chrono::duration_cast< TimeT> - (std::chrono::steady_clock::now() - start); - return duration.count(); - } -}; - -int main(int, char **) -try -{ - auto execute_chain = [](size_t num_threads) - { - std::cerr << "---------------------\n"; - - auto source = std::make_shared(0, 100000); - auto transform1 = std::make_shared(100000); - auto transform2 = std::make_shared(100000); - auto transform3 = std::make_shared(100000); - auto limit = std::make_shared(source->getPort().getHeader(), 20, 0); - auto sink = std::make_shared(""); - - connect(source->getPort(), transform1->getInputPort()); - connect(transform1->getOutputPort(), transform2->getInputPort()); - connect(transform2->getOutputPort(), transform3->getInputPort()); - connect(transform3->getOutputPort(), limit->getInputPort()); - connect(limit->getOutputPort(), sink->getPort()); - - std::vector processors = {source, transform1, transform2, transform3, limit, sink}; -// WriteBufferFromOStream out(std::cout); -// printPipeline(processors, out); - - PipelineExecutor executor(processors); - executor.execute(num_threads); - }; - - auto time_single = Measure<>::execution(execute_chain, 1); - auto time_mt = Measure<>::execution(execute_chain, 4); - - std::cout << "Single Thread time: " << time_single << " ms.\n"; - std::cout << "Multiple Threads time: " << time_mt << " ms.\n"; - - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; -} diff --git a/src/Processors/tests/processors_test_expand_pipeline.cpp b/src/Processors/tests/processors_test_expand_pipeline.cpp deleted file mode 100644 index 83ac2ed0168..00000000000 --- a/src/Processors/tests/processors_test_expand_pipeline.cpp +++ /dev/null @@ -1,285 +0,0 @@ -#include - -#include - -#include -#include -#include -#include -#include - - -#include -#include -#include - -#include - -#include -#include -#include - -using namespace DB; - -class PrintSink : public ISink -{ -public: - String getName() const override { return "Print"; } - - explicit PrintSink(String prefix_) - : ISink(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - prefix(std::move(prefix_)) - { - } - -private: - String prefix; - WriteBufferFromFileDescriptor out{STDOUT_FILENO}; - FormatSettings settings; - - void consume(Chunk chunk) override - { - size_t rows = chunk.getNumRows(); - size_t columns = chunk.getNumColumns(); - - for (size_t row_num = 0; row_num < rows; ++row_num) - { - writeString(prefix, out); - for (size_t column_num = 0; column_num < columns; ++column_num) - { - if (column_num != 0) - writeChar('\t', out); - getPort().getHeader().getByPosition(column_num).type->serializeAsText(*chunk.getColumns()[column_num], row_num, out, settings); - } - writeChar('\n', out); - } - - out.next(); - } -}; - - -class OneNumberSource : public ISource -{ -public: - String getName() const override { return "OneNumber"; } - - explicit OneNumberSource(UInt64 number_) - : ISource(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - number(number_) - { - } - -private: - UInt64 number; - bool done = false; - - Chunk generate() override - { - if (done) - return Chunk(); - - done = true; - - MutableColumns columns; - columns.emplace_back(ColumnUInt64::create(1, number)); - return Chunk(std::move(columns), 1); - } -}; - - -class ExpandingProcessor : public IProcessor -{ -public: - String getName() const override { return "Expanding"; } - ExpandingProcessor() - : IProcessor({Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})}, - {Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})}) - {} - - Status prepare() override - { - auto & main_input = inputs.front(); - auto & main_output = outputs.front(); - auto & additional_input = inputs.back(); - auto & additional_output = outputs.back(); - /// Check can output. - - - if (main_output.isFinished()) - { - main_input.close(); - additional_input.close(); - additional_output.finish(); - return Status::Finished; - } - - if (!main_output.canPush()) - { - main_input.setNotNeeded(); - additional_input.setNotNeeded(); - return Status::PortFull; - } - - if (chunk_from_add_inp && is_processed) - { - if (is_processed) - main_output.push(std::move(chunk_from_add_inp)); - else - return Status::Ready; - } - - if (expanded) - { - if (chunk_from_main_inp) - { - if (additional_output.isFinished()) - { - main_input.close(); - return Status::Finished; - } - - if (!additional_output.canPush()) - { - main_input.setNotNeeded(); - return Status::PortFull; - } - - additional_output.push(std::move(chunk_from_main_inp)); - main_input.close(); - } - - if (additional_input.isFinished()) - { - main_output.finish(); - return Status::Finished; - } - - additional_input.setNeeded(); - - if (!additional_input.hasData()) - return Status::NeedData; - - chunk_from_add_inp = additional_input.pull(); - is_processed = false; - return Status::Ready; - } - else - { - if (!chunk_from_main_inp) - { - - if (main_input.isFinished()) - { - main_output.finish(); - return Status::Finished; - } - - main_input.setNeeded(); - - if (!main_input.hasData()) - return Status::NeedData; - - chunk_from_main_inp = main_input.pull(); - main_input.close(); - } - - UInt64 val = chunk_from_main_inp.getColumns()[0]->getUInt(0); - if (val) - { - --val; - chunk_from_main_inp.setColumns(Columns{ColumnUInt64::create(1, val)}, 1); - return Status::ExpandPipeline; - } - - main_output.push(std::move(chunk_from_main_inp)); - main_output.finish(); - return Status::Finished; - } - } - - Processors expandPipeline() override - { - auto & main_input = inputs.front(); - auto & main_output = outputs.front(); - - Processors processors = {std::make_shared()}; - inputs.push_back({main_input.getHeader(), this}); - outputs.push_back({main_output.getHeader(), this}); - connect(outputs.back(), processors.back()->getInputs().front()); - connect(processors.back()->getOutputs().front(), inputs.back()); - inputs.back().setNeeded(); - - expanded = true; - return processors; - } - - void work() override - { - auto num_rows = chunk_from_add_inp.getNumRows(); - auto columns = chunk_from_add_inp.mutateColumns(); - columns.front()->insert(Field(num_rows)); - chunk_from_add_inp.setColumns(std::move(columns), num_rows + 1); - is_processed = true; - } - -private: - bool expanded = false; - Chunk chunk_from_main_inp; - Chunk chunk_from_add_inp; - bool is_processed = false; -}; - - -template -struct Measure -{ - template - static typename TimeT::rep execution(F&& func, Args&&... args) - { - auto start = std::chrono::steady_clock::now(); - std::forward(func)(std::forward(args)...); - auto duration = std::chrono::duration_cast< TimeT> - (std::chrono::steady_clock::now() - start); - return duration.count(); - } -}; - -int main(int, char **) -try -{ - auto execute = [](String msg, size_t num, size_t num_threads) - { - std::cerr << msg << "\n"; - - auto source = std::make_shared(num); - auto expanding = std::make_shared(); - auto sink = std::make_shared(""); - - connect(source->getPort(), expanding->getInputs().front()); - connect(expanding->getOutputs().front(), sink->getPort()); - - std::vector processors = {source, expanding, sink}; - - PipelineExecutor executor(processors); - executor.execute(num_threads); - - WriteBufferFromOStream out(std::cout); - printPipeline(executor.getProcessors(), out); - }; - - ThreadPool pool(4, 4, 10); - - auto time_single = Measure<>::execution(execute, "Single thread", 10, 1); - auto time_mt = Measure<>::execution(execute, "Multiple threads", 10, 4); - - std::cout << "Single Thread time: " << time_single << " ms.\n"; - std::cout << "Multiple Threads time:" << time_mt << " ms.\n"; - - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; -} diff --git a/src/Processors/tests/processors_test_merge.cpp b/src/Processors/tests/processors_test_merge.cpp deleted file mode 100644 index 11b0bfd1365..00000000000 --- a/src/Processors/tests/processors_test_merge.cpp +++ /dev/null @@ -1,334 +0,0 @@ -#include - -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include - -#include -#include - - -using namespace DB; - - -class MergingSortedProcessor : public IProcessor -{ -public: - MergingSortedProcessor(const Block & header, size_t num_inputs) - : IProcessor(InputPorts(num_inputs, header), OutputPorts{header}) - , chunks(num_inputs), positions(num_inputs, 0), finished(num_inputs, false) - { - } - - String getName() const override { return "MergingSortedProcessor"; } - - Status prepare() override - { - auto & output = outputs.front(); - - /// Check can output. - - if (output.isFinished()) - { - for (auto & in : inputs) - in.close(); - - return Status::Finished; - } - - if (!output.isNeeded()) - { - for (auto & in : inputs) - in.setNotNeeded(); - - return Status::PortFull; - } - - if (output.hasData()) - return Status::PortFull; - - /// Push if has data. - if (res) - { - output.push(std::move(res)); - return Status::PortFull; - } - - /// Check for inputs we need. - bool all_inputs_finished = true; - bool all_inputs_has_data = true; - auto it = inputs.begin(); - for (size_t i = 0; it != inputs.end(); ++it, ++i) - { - auto & input = *it; - if (!finished[i]) - { - if (!input.isFinished()) - { - all_inputs_finished = false; - bool needed = positions[i] >= chunks[i].getNumRows(); - if (needed) - { - input.setNeeded(); - if (input.hasData()) - { - chunks[i] = input.pull(); - positions[i] = 0; - } - else - all_inputs_has_data = false; - } - else - input.setNotNeeded(); - } - else - finished[i] = true; - } - } - - if (all_inputs_finished) - { - output.finish(); - return Status::Finished; - } - - if (!all_inputs_has_data) - return Status::NeedData; - - return Status::Ready; - } - - void work() override - { - using Key = std::pair; - std::priority_queue, std::greater<>> queue; - for (size_t i = 0; i < chunks.size(); ++i) - { - if (finished[i]) - continue; - - if (positions[i] >= chunks[i].getNumRows()) - return; - - queue.push({chunks[i].getColumns()[0]->getUInt(positions[i]), i}); - } - - auto col = ColumnUInt64::create(); - - while (!queue.empty()) - { - size_t ps = queue.top().second; - queue.pop(); - - const auto & cur_col = chunks[ps].getColumns()[0]; - col->insertFrom(*cur_col, positions[ps]); - ++positions[ps]; - - if (positions[ps] == cur_col->size()) - break; - - queue.push({cur_col->getUInt(positions[ps]), ps}); - } - - UInt64 num_rows = col->size(); - res.setColumns(Columns({std::move(col)}), num_rows); - } - - OutputPort & getOutputPort() { return outputs.front(); } - -private: - Chunks chunks; - Chunk res; - std::vector positions; - std::vector finished; -}; - - -class NumbersSource : public ISource -{ -public: - String getName() const override { return "Numbers"; } - - NumbersSource(UInt64 start_number, UInt64 step_, unsigned sleep_useconds_) - : ISource(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - current_number(start_number), step(step_), sleep_useconds(sleep_useconds_) - { - } - -private: - UInt64 current_number = 0; - UInt64 step; - unsigned sleep_useconds; - - Chunk generate() override - { - usleep(sleep_useconds); - - MutableColumns columns; - columns.emplace_back(ColumnUInt64::create(1, current_number)); - current_number += step; - return Chunk(std::move(columns), 1); - } -}; - - -class SleepyTransform : public ISimpleTransform -{ -public: - explicit SleepyTransform(unsigned sleep_useconds_) - : ISimpleTransform( - Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }}), - Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }}), - false) - , sleep_useconds(sleep_useconds_) {} - - String getName() const override { return "SleepyTransform"; } - -protected: - void transform(Chunk &) override - { - usleep(sleep_useconds); - } - -private: - unsigned sleep_useconds; -}; - -class PrintSink : public ISink -{ -public: - String getName() const override { return "Print"; } - - explicit PrintSink(String prefix_) - : ISink(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - prefix(std::move(prefix_)) - { - } - -private: - String prefix; - WriteBufferFromFileDescriptor out{STDOUT_FILENO}; - FormatSettings settings; - - void consume(Chunk chunk) override - { - size_t rows = chunk.getNumRows(); - size_t columns = chunk.getNumColumns(); - - for (size_t row_num = 0; row_num < rows; ++row_num) - { - writeString(prefix, out); - for (size_t column_num = 0; column_num < columns; ++column_num) - { - if (column_num != 0) - writeChar('\t', out); - getPort().getHeader().getByPosition(column_num).type->serializeAsText(*chunk.getColumns()[column_num], row_num, out, settings); - } - writeChar('\n', out); - } - - out.next(); - } -}; - -template -struct Measure -{ - template - static typename TimeT::rep execution(F&& func, Args&&... args) - { - auto start = std::chrono::steady_clock::now(); - std::forward(func)(std::forward(args)...); - auto duration = std::chrono::duration_cast< TimeT> - (std::chrono::steady_clock::now() - start); - return duration.count(); - } -}; - -int main(int, char **) -try -{ - auto execute_chain = [](String msg, size_t start1, size_t start2, size_t start3, size_t num_threads) - { - std::cerr << msg << "\n"; - - auto source1 = std::make_shared(start1, 3, 100000); - auto source2 = std::make_shared(start2, 3, 100000); - auto source3 = std::make_shared(start3, 3, 100000); - - auto transform1 = std::make_shared(100000); - auto transform2 = std::make_shared(100000); - auto transform3 = std::make_shared(100000); - - auto limit1 = std::make_shared(source1->getPort().getHeader(), 20, 0); - auto limit2 = std::make_shared(source2->getPort().getHeader(), 20, 0); - auto limit3 = std::make_shared(source3->getPort().getHeader(), 20, 0); - - auto merge = std::make_shared(source1->getPort().getHeader(), 3); - auto limit_fin = std::make_shared(source1->getPort().getHeader(), 54, 0); - auto sink = std::make_shared(""); - - connect(source1->getPort(), transform1->getInputPort()); - connect(source2->getPort(), transform2->getInputPort()); - connect(source3->getPort(), transform3->getInputPort()); - - connect(transform1->getOutputPort(), limit1->getInputPort()); - connect(transform2->getOutputPort(), limit2->getInputPort()); - connect(transform3->getOutputPort(), limit3->getInputPort()); - - auto it = merge->getInputs().begin(); - connect(limit1->getOutputPort(), *(it++)); - connect(limit2->getOutputPort(), *(it++)); - connect(limit3->getOutputPort(), *(it++)); - - connect(merge->getOutputPort(), limit_fin->getInputPort()); - connect(limit_fin->getOutputPort(), sink->getPort()); - - std::vector processors = {source1, source2, source3, - transform1, transform2, transform3, - limit1, limit2, limit3, - merge, limit_fin, sink}; -// WriteBufferFromOStream out(std::cout); -// printPipeline(processors, out); - - PipelineExecutor executor(processors); - executor.execute(num_threads); - }; - - auto even_time_single = Measure<>::execution(execute_chain, "Even distribution single thread", 0, 1, 2, 1); - auto even_time_mt = Measure<>::execution(execute_chain, "Even distribution multiple threads", 0, 1, 2, 4); - - auto half_time_single = Measure<>::execution(execute_chain, "Half distribution single thread", 0, 31, 62, 1); - auto half_time_mt = Measure<>::execution(execute_chain, "Half distribution multiple threads", 0, 31, 62, 4); - - auto ordered_time_single = Measure<>::execution(execute_chain, "Ordered distribution single thread", 0, 61, 122, 1); - auto ordered_time_mt = Measure<>::execution(execute_chain, "Ordered distribution multiple threads", 0, 61, 122, 4); - - std::cout << "Single Thread [0:60:3] [1:60:3] [2:60:3] time: " << even_time_single << " ms.\n"; - std::cout << "Multiple Threads [0:60:3] [1:60:3] [2:60:3] time:" << even_time_mt << " ms.\n"; - - std::cout << "Single Thread [0:60:3] [31:90:3] [62:120:3] time: " << half_time_single << " ms.\n"; - std::cout << "Multiple Threads [0:60:3] [31:90:3] [62:120:3] time: " << half_time_mt << " ms.\n"; - - std::cout << "Single Thread [0:60:3] [61:120:3] [122:180:3] time: " << ordered_time_single << " ms.\n"; - std::cout << "Multiple Threads [0:60:3] [61:120:3] [122:180:3] time: " << ordered_time_mt << " ms.\n"; - - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; -} diff --git a/src/Processors/tests/processors_test_merge_sorting_transform.cpp b/src/Processors/tests/processors_test_merge_sorting_transform.cpp deleted file mode 100644 index 5e6720f0167..00000000000 --- a/src/Processors/tests/processors_test_merge_sorting_transform.cpp +++ /dev/null @@ -1,250 +0,0 @@ -#include - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include - -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -class NumbersSource : public ISource -{ -public: - String getName() const override { return "Numbers"; } - - NumbersSource(UInt64 count_, UInt64 block_size_, unsigned sleep_useconds_) - : ISource(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - count(count_), block_size(block_size_), sleep_useconds(sleep_useconds_) - { - } - -private: - UInt64 current_number = 0; - UInt64 count; - UInt64 block_size; - unsigned sleep_useconds; - - Chunk generate() override - { - if (current_number == count) - return {}; - - usleep(sleep_useconds); - - MutableColumns columns; - columns.emplace_back(ColumnUInt64::create()); - - UInt64 number = current_number++; - for (UInt64 i = 0; i < block_size; ++i, number += count) - columns.back()->insert(Field(number)); - - return Chunk(std::move(columns), block_size); - } -}; - -class CheckSortedSink : public ISink -{ -public: - String getName() const override { return "Print"; } - - CheckSortedSink() - : ISink(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})) - { - } - -private: - FormatSettings settings; - UInt64 current_number = 0; - - void consume(Chunk chunk) override - { - size_t rows = chunk.getNumRows(); - - UInt64 prev = current_number; - const auto & col = chunk.getColumns().at(0); - for (size_t row_num = 0; row_num < rows; ++row_num) - { - UInt64 val = col->getUInt(row_num); - if (val != current_number) - throw Exception("Invalid value. Expected " + toString(current_number) + ", got " + toString(val), - ErrorCodes::LOGICAL_ERROR); - - ++current_number; - } - - std::cout << "CheckSortedSink: " << prev << " - " << current_number << std::endl; - } -}; - -template -struct Measure -{ - template - static typename TimeT::rep execution(F&& func, Args&&... args) - { - auto start = std::chrono::steady_clock::now(); - std::forward(func)(std::forward(args)...); - auto duration = std::chrono::duration_cast< TimeT> - (std::chrono::steady_clock::now() - start); - return duration.count(); - } -}; - -} - - -using namespace DB; - -int main(int, char **) -try -{ - Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Poco::Logger::root().setChannel(channel); - Poco::Logger::root().setLevel("trace"); - - auto disk = std::make_shared("tmp", ".", 0); - auto tmp_volume = std::make_shared("tmp", std::vector{disk}, 0); - - auto execute_chain = [tmp_volume]( - String msg, - UInt64 source_block_size, - UInt64 blocks_count, - size_t max_merged_block_size, - UInt64 limit, - size_t max_bytes_before_remerge, - size_t max_bytes_before_external_sort, - size_t num_threads) - { - std::cerr << "------------------------\n"; - std::cerr << msg << "\n"; - - auto source = std::make_shared(blocks_count, source_block_size, 100); - SortDescription description = {{0, 1, 1}}; - auto transform = std::make_shared( - source->getPort().getHeader(), description, - max_merged_block_size, limit, - max_bytes_before_remerge, max_bytes_before_external_sort, - tmp_volume, 0); - auto sink = std::make_shared(); - - connect(source->getPort(), transform->getInputs().front()); - connect(transform->getOutputs().front(), sink->getPort()); - - std::vector processors = {source, transform, sink}; - PipelineExecutor executor(processors); - executor.execute(num_threads); - - WriteBufferFromOStream out(std::cout); - printPipeline(executor.getProcessors(), out); - }; - - std::map times; - - for (size_t num_threads : {1, 4}) - { - { - UInt64 source_block_size = 100; - UInt64 blocks_count = 10; - size_t max_merged_block_size = 100; - UInt64 limit = 0; - size_t max_bytes_before_remerge = 10000000; - size_t max_bytes_before_external_sort = 10000000; - std::string msg = num_threads > 1 ? "multiple threads" : "single thread"; - msg += ", " + toString(blocks_count) + " blocks per " + toString(source_block_size) + " numbers" + - ", no remerge and external sorts."; - - Int64 time = Measure<>::execution(execute_chain, msg, - source_block_size, - blocks_count, - max_merged_block_size, - limit, - max_bytes_before_remerge, - max_bytes_before_external_sort, - num_threads); - - times[msg] = time; - } - - { - UInt64 source_block_size = 1024; - UInt64 blocks_count = 10; - size_t max_merged_block_size = 1024; - UInt64 limit = 2048; - size_t max_bytes_before_remerge = sizeof(UInt64) * source_block_size * 4; - size_t max_bytes_before_external_sort = 10000000; - std::string msg = num_threads > 1 ? "multiple threads" : "single thread"; - msg += ", " + toString(blocks_count) + " blocks per " + toString(source_block_size) + " numbers" + - ", with remerge, no external sorts."; - - Int64 time = Measure<>::execution(execute_chain, msg, - source_block_size, - blocks_count, - max_merged_block_size, - limit, - max_bytes_before_remerge, - max_bytes_before_external_sort, - num_threads); - - times[msg] = time; - } - - { - UInt64 source_block_size = 1024; - UInt64 blocks_count = 10; - size_t max_merged_block_size = 1024; - UInt64 limit = 0; - size_t max_bytes_before_remerge = 0; - size_t max_bytes_before_external_sort = sizeof(UInt64) * source_block_size * 4; - std::string msg = num_threads > 1 ? "multiple threads" : "single thread"; - msg += ", " + toString(blocks_count) + " blocks per " + toString(source_block_size) + " numbers" + - ", no remerge, with external sorts."; - - Int64 time = Measure<>::execution(execute_chain, msg, - source_block_size, - blocks_count, - max_merged_block_size, - limit, - max_bytes_before_remerge, - max_bytes_before_external_sort, - num_threads); - - times[msg] = time; - } - } - - for (auto & item : times) - std::cout << item.first << ' ' << item.second << " ms.\n"; - - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; -} diff --git a/src/Processors/tests/processors_test_merging_sorted_transform.cpp b/src/Processors/tests/processors_test_merging_sorted_transform.cpp deleted file mode 100644 index 477626d165d..00000000000 --- a/src/Processors/tests/processors_test_merging_sorted_transform.cpp +++ /dev/null @@ -1,207 +0,0 @@ -#include - -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include - -#include -#include - - -using namespace DB; - - -class NumbersSource : public ISource -{ -public: - String getName() const override { return "Numbers"; } - - NumbersSource(UInt64 start_number, UInt64 step_, UInt64 block_size_, unsigned sleep_useconds_) - : ISource(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - current_number(start_number), step(step_), block_size(block_size_), sleep_useconds(sleep_useconds_) - { - } - -private: - UInt64 current_number = 0; - UInt64 step; - UInt64 block_size; - unsigned sleep_useconds; - - Chunk generate() override - { - usleep(sleep_useconds); - - MutableColumns columns; - columns.emplace_back(ColumnUInt64::create()); - - for (UInt64 i = 0; i < block_size; ++i, current_number += step) - columns.back()->insert(Field(current_number)); - - return Chunk(std::move(columns), block_size); - } -}; - - -class SleepyTransform : public ISimpleTransform -{ -public: - explicit SleepyTransform(unsigned sleep_useconds_) - : ISimpleTransform( - Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }}), - Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }}), - false) - , sleep_useconds(sleep_useconds_) {} - - String getName() const override { return "SleepyTransform"; } - -protected: - void transform(Chunk &) override - { - usleep(sleep_useconds); - } - -private: - unsigned sleep_useconds; -}; - -class PrintSink : public ISink -{ -public: - String getName() const override { return "Print"; } - - explicit PrintSink(String prefix_) - : ISink(Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared(), "number" }})), - prefix(std::move(prefix_)) - { - } - -private: - String prefix; - WriteBufferFromFileDescriptor out{STDOUT_FILENO}; - FormatSettings settings; - - void consume(Chunk chunk) override - { - size_t rows = chunk.getNumRows(); - size_t columns = chunk.getNumColumns(); - - for (size_t row_num = 0; row_num < rows; ++row_num) - { - writeString(prefix, out); - for (size_t column_num = 0; column_num < columns; ++column_num) - { - if (column_num != 0) - writeChar('\t', out); - getPort().getHeader().getByPosition(column_num).type->serializeAsText(*chunk.getColumns()[column_num], row_num, out, settings); - } - writeChar('\n', out); - } - - out.next(); - } -}; - -template -struct Measure -{ - template - static typename TimeT::rep execution(F&& func, Args&&... args) - { - auto start = std::chrono::steady_clock::now(); - std::forward(func)(std::forward(args)...); - auto duration = std::chrono::duration_cast< TimeT> - (std::chrono::steady_clock::now() - start); - return duration.count(); - } -}; - -int main(int, char **) -try -{ - auto execute_chain = [](String msg, size_t start1, size_t start2, size_t start3, size_t num_threads) - { - std::cerr << msg << "\n"; - - auto source1 = std::make_shared(start1, 3, 2, 100000); - auto source2 = std::make_shared(start2, 3, 2, 100000); - auto source3 = std::make_shared(start3, 3, 2, 100000); - - auto transform1 = std::make_shared(100000); - auto transform2 = std::make_shared(100000); - auto transform3 = std::make_shared(100000); - - auto limit1 = std::make_shared(source1->getPort().getHeader(), 20, 0); - auto limit2 = std::make_shared(source2->getPort().getHeader(), 20, 0); - auto limit3 = std::make_shared(source3->getPort().getHeader(), 20, 0); - - SortDescription description = {{0, 1, 1}}; - auto merge = std::make_shared(source1->getPort().getHeader(), 3, description, 2); - auto limit_fin = std::make_shared(source1->getPort().getHeader(), 54, 0); - auto sink = std::make_shared(""); - - connect(source1->getPort(), transform1->getInputPort()); - connect(source2->getPort(), transform2->getInputPort()); - connect(source3->getPort(), transform3->getInputPort()); - - connect(transform1->getOutputPort(), limit1->getInputPort()); - connect(transform2->getOutputPort(), limit2->getInputPort()); - connect(transform3->getOutputPort(), limit3->getInputPort()); - - auto it = merge->getInputs().begin(); - connect(limit1->getOutputPort(), *(it++)); - connect(limit2->getOutputPort(), *(it++)); - connect(limit3->getOutputPort(), *(it++)); - - connect(merge->getOutputs().front(), limit_fin->getInputPort()); - connect(limit_fin->getOutputPort(), sink->getPort()); - - std::vector processors = {source1, source2, source3, - transform1, transform2, transform3, - limit1, limit2, limit3, - merge, limit_fin, sink}; -// WriteBufferFromOStream out(std::cout); -// printPipeline(processors, out); - - PipelineExecutor executor(processors); - executor.execute(num_threads); - }; - - auto even_time_single = Measure<>::execution(execute_chain, "Even distribution single thread", 0, 1, 2, 1); - auto even_time_mt = Measure<>::execution(execute_chain, "Even distribution multiple threads", 0, 1, 2, 4); - - auto half_time_single = Measure<>::execution(execute_chain, "Half distribution single thread", 0, 31, 62, 1); - auto half_time_mt = Measure<>::execution(execute_chain, "Half distribution multiple threads", 0, 31, 62, 4); - - auto ordered_time_single = Measure<>::execution(execute_chain, "Ordered distribution single thread", 0, 61, 122, 1); - auto ordered_time_mt = Measure<>::execution(execute_chain, "Ordered distribution multiple threads", 0, 61, 122, 4); - - std::cout << "Single Thread [0:60:3] [1:60:3] [2:60:3] time: " << even_time_single << " ms.\n"; - std::cout << "Multiple Threads [0:60:3] [1:60:3] [2:60:3] time:" << even_time_mt << " ms.\n"; - - std::cout << "Single Thread [0:60:3] [31:90:3] [62:120:3] time: " << half_time_single << " ms.\n"; - std::cout << "Multiple Threads [0:60:3] [31:90:3] [62:120:3] time: " << half_time_mt << " ms.\n"; - - std::cout << "Single Thread [0:60:3] [61:120:3] [122:180:3] time: " << ordered_time_single << " ms.\n"; - std::cout << "Multiple Threads [0:60:3] [61:120:3] [122:180:3] time: " << ordered_time_mt << " ms.\n"; - - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; -} diff --git a/src/Storages/Kafka/KafkaBlockInputStream.cpp b/src/Storages/Kafka/KafkaBlockInputStream.cpp index 3e4533f8bb2..3edfcc7b9d2 100644 --- a/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } KafkaBlockInputStream::KafkaBlockInputStream( - StorageKafka & storage_, const Context & context_, const Names & columns, size_t max_block_size_, bool commit_in_suffix_) + StorageKafka & storage_, const std::shared_ptr & context_, const Names & columns, size_t max_block_size_, bool commit_in_suffix_) : storage(storage_) , context(context_) , column_names(columns) @@ -22,12 +22,6 @@ KafkaBlockInputStream::KafkaBlockInputStream( , non_virtual_header(storage.getSampleBlockNonMaterialized()) , virtual_header(storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp","_timestamp_ms","_headers.name","_headers.value"})) { - context.setSetting("input_format_skip_unknown_fields", 1u); // Always skip unknown fields regardless of the context (JSON or TSKV) - context.setSetting("input_format_allow_errors_ratio", 0.); - context.setSetting("input_format_allow_errors_num", storage.skipBroken()); - - if (!storage.getSchemaName().empty()) - context.setSetting("format_schema", storage.getSchemaName()); } KafkaBlockInputStream::~KafkaBlockInputStream() @@ -48,7 +42,7 @@ Block KafkaBlockInputStream::getHeader() const void KafkaBlockInputStream::readPrefixImpl() { - auto timeout = std::chrono::milliseconds(context.getSettingsRef().kafka_max_wait_ms.totalMilliseconds()); + auto timeout = std::chrono::milliseconds(context->getSettingsRef().kafka_max_wait_ms.totalMilliseconds()); buffer = storage.popReadBuffer(timeout); if (!buffer) @@ -73,7 +67,7 @@ Block KafkaBlockInputStream::readImpl() MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + storage.getFormatName(), *buffer, non_virtual_header, *context, max_block_size); InputPort port(input_format->getPort().getHeader(), input_format.get()); connect(input_format->getPort(), port); diff --git a/src/Storages/Kafka/KafkaBlockInputStream.h b/src/Storages/Kafka/KafkaBlockInputStream.h index e3052122894..387f5088721 100644 --- a/src/Storages/Kafka/KafkaBlockInputStream.h +++ b/src/Storages/Kafka/KafkaBlockInputStream.h @@ -14,7 +14,7 @@ class KafkaBlockInputStream : public IBlockInputStream { public: KafkaBlockInputStream( - StorageKafka & storage_, const Context & context_, const Names & columns, size_t max_block_size_, bool commit_in_suffix = true); + StorageKafka & storage_, const std::shared_ptr & context_, const Names & columns, size_t max_block_size_, bool commit_in_suffix = true); ~KafkaBlockInputStream() override; String getName() const override { return storage.getName(); } @@ -29,7 +29,7 @@ public: private: StorageKafka & storage; - Context context; + const std::shared_ptr context; Names column_names; UInt64 max_block_size; diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index fe8aa207c93..17ef5aa104c 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes extern const int CANNOT_CREATE_IO_BUFFER; } -KafkaBlockOutputStream::KafkaBlockOutputStream(StorageKafka & storage_, const Context & context_) : storage(storage_), context(context_) +KafkaBlockOutputStream::KafkaBlockOutputStream(StorageKafka & storage_, const std::shared_ptr & context_) : storage(storage_), context(context_) { } @@ -26,7 +26,7 @@ void KafkaBlockOutputStream::writePrefix() if (!buffer) throw Exception("Failed to create Kafka producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); - child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, getHeader(), context, [this](const Columns & columns, size_t row){ buffer->countRow(columns, row); }); + child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, getHeader(), *context, [this](const Columns & columns, size_t row){ buffer->countRow(columns, row); }); } void KafkaBlockOutputStream::write(const Block & block) diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.h b/src/Storages/Kafka/KafkaBlockOutputStream.h index f3eb3dae0ba..7a973724f1b 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.h +++ b/src/Storages/Kafka/KafkaBlockOutputStream.h @@ -10,7 +10,7 @@ namespace DB class KafkaBlockOutputStream : public IBlockOutputStream { public: - explicit KafkaBlockOutputStream(StorageKafka & storage_, const Context & context_); + explicit KafkaBlockOutputStream(StorageKafka & storage_, const std::shared_ptr & context_); Block getHeader() const override; @@ -22,7 +22,7 @@ public: private: StorageKafka & storage; - Context context; + const std::shared_ptr context; ProducerBufferPtr buffer; BlockOutputStreamPtr child; }; diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 43984f81e05..e65522b3606 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -1,7 +1,7 @@ #pragma once #include - +#include namespace DB { @@ -15,18 +15,34 @@ struct KafkaSettings : public SettingsCollection { -#define LIST_OF_KAFKA_SETTINGS(M) \ +#define KAFKA_RELATED_SETTINGS(M) \ M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \ M(SettingString, kafka_topic_list, "", "A list of Kafka topics.", 0) \ - M(SettingString, kafka_group_name, "", "A group of Kafka consumers.", 0) \ - M(SettingString, kafka_client_id, "", "A client id of Kafka consumer.", 0) \ + M(SettingString, kafka_group_name, "", "Client group id string. All Kafka consumers sharing the same group.id belong to the same group.", 0) \ + M(SettingString, kafka_client_id, "", "Client identifier.", 0) \ + M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \ + M(SettingBool, kafka_commit_every_batch, false, "Commit every consumed and handled batch instead of a single commit after writing a whole block", 0) \ + /* default is stream_poll_timeout_ms */ \ + M(SettingMilliseconds, kafka_poll_timeout_ms, 0, "Timeout for single poll from Kafka.", 0) \ + /* default is min(max_block_size, kafka_max_block_size)*/ \ + M(SettingUInt64, kafka_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \ + /* default is = min_insert_block_size / kafka_num_consumers */ \ + M(SettingUInt64, kafka_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \ + /* default is stream_flush_interval_ms */ \ + M(SettingMilliseconds, kafka_flush_interval_ms, 0, "Timeout for flushing data from Kafka.", 0) \ + /* those are mapped to format factory settings */ \ M(SettingString, kafka_format, "", "The message format for Kafka engine.", 0) \ M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.", 0) \ M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ - M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \ - M(SettingUInt64, kafka_max_block_size, 0, "The maximum batch size for poll.", 0) \ - M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) \ - M(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block", 0) + M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) + + /** TODO: */ + /* https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md */ + /* https://github.com/edenhill/librdkafka/blob/v1.4.2/src/rdkafka_conf.c */ + +#define LIST_OF_KAFKA_SETTINGS(M) \ + KAFKA_RELATED_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) DECLARE_SETTINGS_COLLECTION(LIST_OF_KAFKA_SETTINGS) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index fc83fd84884..bb721417c5b 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -119,39 +119,74 @@ StorageKafka::StorageKafka( const StorageID & table_id_, Context & context_, const ColumnsDescription & columns_, - const String & brokers_, - const String & group_, - const String & client_id_, - const Names & topics_, - const String & format_name_, - char row_delimiter_, - const String & schema_name_, - size_t num_consumers_, - UInt64 max_block_size_, - size_t skip_broken_, - bool intermediate_commit_) + std::unique_ptr kafka_settings_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) - , kafka_context(Context(global_context)) - , topics(global_context.getMacros()->expand(topics_)) - , brokers(global_context.getMacros()->expand(brokers_)) - , group(global_context.getMacros()->expand(group_)) - , client_id(client_id_.empty() ? getDefaultClientId(table_id_) : global_context.getMacros()->expand(client_id_)) - , format_name(global_context.getMacros()->expand(format_name_)) - , row_delimiter(row_delimiter_) - , schema_name(global_context.getMacros()->expand(schema_name_)) - , num_consumers(num_consumers_) - , max_block_size(max_block_size_) + , kafka_context(std::make_shared(global_context)) + , kafka_settings(std::move(kafka_settings_)) + , topics(parseTopics(global_context.getMacros()->expand(kafka_settings->kafka_topic_list.value))) + , brokers(global_context.getMacros()->expand(kafka_settings->kafka_broker_list.value)) + , group(global_context.getMacros()->expand(kafka_settings->kafka_group_name.value)) + , client_id(kafka_settings->kafka_client_id.value.empty() ? getDefaultClientId(table_id_) : global_context.getMacros()->expand(kafka_settings->kafka_client_id.value)) + , format_name(global_context.getMacros()->expand(kafka_settings->kafka_format.value)) + , row_delimiter(kafka_settings->kafka_row_delimiter.value) + , schema_name(global_context.getMacros()->expand(kafka_settings->kafka_schema.value)) + , num_consumers(kafka_settings->kafka_num_consumers.value) , log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")")) - , semaphore(0, num_consumers_) - , skip_broken(skip_broken_) - , intermediate_commit(intermediate_commit_) + , semaphore(0, num_consumers) + , intermediate_commit(kafka_settings->kafka_commit_every_batch.value) + , settings_adjustments(createSettingsAdjustments()) { - kafka_context.makeQueryContext(); - setColumns(columns_); task = global_context.getSchedulePool().createTask(log->name(), [this]{ threadFunc(); }); task->deactivate(); + + kafka_context->makeQueryContext(); + kafka_context->applySettingsChanges(settings_adjustments); +} + +SettingsChanges StorageKafka::createSettingsAdjustments() +{ + SettingsChanges result; + // Needed for backward compatibility + if (!kafka_settings->input_format_skip_unknown_fields.changed) + { + // Always skip unknown fields regardless of the context (JSON or TSKV) + kafka_settings->input_format_skip_unknown_fields = true; + } + + if (!kafka_settings->input_format_allow_errors_ratio.changed) + { + kafka_settings->input_format_allow_errors_ratio = 0.; + } + + if (!kafka_settings->input_format_allow_errors_num.changed) + { + kafka_settings->input_format_allow_errors_num = kafka_settings->kafka_skip_broken_messages.value; + } + + if (!schema_name.empty()) + result.emplace_back("format_schema", schema_name); + + for (auto & it : *kafka_settings) + { + if (it.isChanged() && it.getName().toString().rfind("kafka_",0) == std::string::npos) + { + result.emplace_back(it.getName().toString(), it.getValueAsString()); + } + } + return result; +} + +Names StorageKafka::parseTopics(String topic_list) +{ + Names result; + boost::split(result,topic_list,[](char c){ return c == ','; }); + for (String & topic : result) + { + boost::trim(topic); + } + return result; } String StorageKafka::getDefaultClientId(const StorageID & table_id_) @@ -176,6 +211,8 @@ Pipes StorageKafka::read( /// Always use all consumers at once, otherwise SELECT may not read messages from all partitions. Pipes pipes; pipes.reserve(num_created_consumers); + auto modified_context = std::make_shared(context); + modified_context->applySettingsChanges(settings_adjustments); // Claim as many consumers as requested, but don't block for (size_t i = 0; i < num_created_consumers; ++i) @@ -184,7 +221,7 @@ Pipes StorageKafka::read( /// TODO: probably that leads to awful performance. /// FIXME: seems that doesn't help with extra reading and committing unprocessed messages. /// TODO: rewrite KafkaBlockInputStream to KafkaSource. Now it is used in other place. - pipes.emplace_back(std::make_shared(std::make_shared(*this, context, column_names, 1))); + pipes.emplace_back(std::make_shared(std::make_shared(*this, modified_context, column_names, 1))); } LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); @@ -194,9 +231,12 @@ Pipes StorageKafka::read( BlockOutputStreamPtr StorageKafka::write(const ASTPtr &, const Context & context) { + auto modified_context = std::make_shared(context); + modified_context->applySettingsChanges(settings_adjustments); + if (topics.size() > 1) throw Exception("Can't write to Kafka table with multiple topics!", ErrorCodes::NOT_IMPLEMENTED); - return std::make_shared(*this, context); + return std::make_shared(*this, modified_context); } @@ -268,13 +308,14 @@ ConsumerBufferPtr StorageKafka::popReadBuffer(std::chrono::milliseconds timeout) return buffer; } - ProducerBufferPtr StorageKafka::createWriteBuffer(const Block & header) { cppkafka::Configuration conf; conf.set("metadata.broker.list", brokers); conf.set("group.id", group); conf.set("client.id", client_id); + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); // TODO: fill required settings updateConfiguration(conf); @@ -303,9 +344,16 @@ ConsumerBufferPtr StorageKafka::createReadBuffer(const size_t consumer_number) { conf.set("client.id", client_id); } - + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); conf.set("auto.offset.reset", "smallest"); // If no offset stored for this group, read all messages from the start + // that allows to prevent fast draining of the librdkafka queue + // during building of single insert block. Improves performance + // significantly, but may lead to bigger memory consumption. + size_t default_queued_min_messages = 100000; // we don't want to decrease the default + conf.set("queued.min.messages", std::max(getMaxBlockSize(),default_queued_min_messages)); + updateConfiguration(conf); // those settings should not be changed by users. @@ -317,17 +365,32 @@ ConsumerBufferPtr StorageKafka::createReadBuffer(const size_t consumer_number) auto consumer = std::make_shared(conf); consumer->set_destroy_flags(RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); - // Limit the number of batched messages to allow early cancellations - const Settings & settings = global_context.getSettingsRef(); - size_t batch_size = max_block_size; - if (!batch_size) - batch_size = settings.max_block_size.value; - size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); - /// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage. - return std::make_shared(consumer, log, batch_size, poll_timeout, intermediate_commit, stream_cancelled, getTopics()); + return std::make_shared(consumer, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, stream_cancelled, topics); } +size_t StorageKafka::getMaxBlockSize() const +{ + return kafka_settings->kafka_max_block_size.changed + ? kafka_settings->kafka_max_block_size.value + : (global_context.getSettingsRef().max_insert_block_size.value / num_consumers); +} + +size_t StorageKafka::getPollMaxBatchSize() const +{ + size_t batch_size = kafka_settings->kafka_poll_max_batch_size.changed + ? kafka_settings->kafka_poll_max_batch_size.value + : global_context.getSettingsRef().max_block_size.value; + + return std::min(batch_size,getMaxBlockSize()); +} + +size_t StorageKafka::getPollTimeoutMillisecond() const +{ + return kafka_settings->kafka_poll_timeout_ms.changed + ? kafka_settings->kafka_poll_timeout_ms.totalMilliseconds() + : global_context.getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); +} void StorageKafka::updateConfiguration(cppkafka::Configuration & conf) { @@ -458,19 +521,17 @@ bool StorageKafka::streamToViews() auto insert = std::make_shared(); insert->table_id = table_id; - const Settings & settings = global_context.getSettingsRef(); - size_t block_size = max_block_size; - if (block_size == 0) - block_size = settings.max_block_size; + size_t block_size = getMaxBlockSize(); // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, kafka_context, false, true, true); + InterpreterInsertQuery interpreter(insert, *kafka_context, false, true, true); auto block_io = interpreter.execute(); // Create a stream for each consumer and join them in a union stream BlockInputStreams streams; streams.reserve(num_created_consumers); + for (size_t i = 0; i < num_created_consumers; ++i) { auto stream @@ -479,7 +540,11 @@ bool StorageKafka::streamToViews() // Limit read batch to maximum block size to allow DDL IBlockInputStream::LocalLimits limits; - limits.speed_limits.max_execution_time = settings.stream_flush_interval_ms; + + limits.speed_limits.max_execution_time = kafka_settings->kafka_flush_interval_ms.changed + ? kafka_settings->kafka_flush_interval_ms + : global_context.getSettingsRef().stream_flush_interval_ms; + limits.timeout_overflow_mode = OverflowMode::BREAK; stream->setLimits(limits); } @@ -514,17 +579,61 @@ void registerStorageKafka(StorageFactory & factory) size_t args_count = engine_args.size(); bool has_settings = args.storage_def->settings; - KafkaSettings kafka_settings; + auto kafka_settings = std::make_unique(); if (has_settings) { - kafka_settings.loadFromQuery(*args.storage_def); + kafka_settings->loadFromQuery(*args.storage_def); } + // Check arguments and settings + #define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME, EVAL) \ + /* One of the four required arguments is not specified */ \ + if (args_count < (ARG_NUM) && (ARG_NUM) <= 4 && \ + !kafka_settings->PAR_NAME.changed) \ + { \ + throw Exception( \ + "Required parameter '" #PAR_NAME "' " \ + "for storage Kafka not specified", \ + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); \ + } \ + if (args_count >= (ARG_NUM)) \ + { \ + /* The same argument is given in two places */ \ + if (has_settings && \ + kafka_settings->PAR_NAME.changed) \ + { \ + throw Exception( \ + "The argument №" #ARG_NUM " of storage Kafka " \ + "and the parameter '" #PAR_NAME "' " \ + "in SETTINGS cannot be specified at the same time", \ + ErrorCodes::BAD_ARGUMENTS); \ + } \ + /* move engine args to settings */ \ + else \ + { \ + if ((EVAL) == 1) \ + { \ + engine_args[(ARG_NUM)-1] = \ + evaluateConstantExpressionAsLiteral( \ + engine_args[(ARG_NUM)-1], \ + args.local_context); \ + } \ + if ((EVAL) == 2) \ + { \ + engine_args[(ARG_NUM)-1] = \ + evaluateConstantExpressionOrIdentifierAsLiteral( \ + engine_args[(ARG_NUM)-1], \ + args.local_context); \ + } \ + kafka_settings->PAR_NAME.set( \ + engine_args[(ARG_NUM)-1]->as().value);\ + } \ + } + /** Arguments of engine is following: * - Kafka broker list * - List of topics * - Group ID (may be a constaint expression with a string result) - * - Client ID * - Message format (string) * - Row delimiter * - Schema (optional, if the format supports it) @@ -534,209 +643,32 @@ void registerStorageKafka(StorageFactory & factory) * - Do intermediate commits when the batch consumed and handled */ - // Check arguments and settings - #define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME) \ - /* One of the four required arguments is not specified */ \ - if (args_count < (ARG_NUM) && (ARG_NUM) <= 4 && \ - !kafka_settings.PAR_NAME.changed) \ - { \ - throw Exception( \ - "Required parameter '" #PAR_NAME "' " \ - "for storage Kafka not specified", \ - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); \ - } \ - /* The same argument is given in two places */ \ - if (has_settings && \ - kafka_settings.PAR_NAME.changed && \ - args_count >= (ARG_NUM)) \ - { \ - throw Exception( \ - "The argument №" #ARG_NUM " of storage Kafka " \ - "and the parameter '" #PAR_NAME "' " \ - "in SETTINGS cannot be specified at the same time", \ - ErrorCodes::BAD_ARGUMENTS); \ - } - - CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list) - CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list) - CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name) - CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format) - CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter) - CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema) - CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers) - CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size) - CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages) - CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch) + /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ + CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) + CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) #undef CHECK_KAFKA_STORAGE_ARGUMENT - // Get and check broker list - String brokers = kafka_settings.kafka_broker_list; - if (args_count >= 1) + auto num_consumers = kafka_settings->kafka_num_consumers.value; + + if (num_consumers > 16) { - const auto * ast = engine_args[0]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - brokers = safeGet(ast->value); - } - else - { - throw Exception(String("Kafka broker list must be a string"), ErrorCodes::BAD_ARGUMENTS); - } + throw Exception("Number of consumers can not be bigger than 16", ErrorCodes::BAD_ARGUMENTS); + } + else if (num_consumers < 1) + { + throw Exception("Number of consumers can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } - // Get and check topic list - String topic_list = kafka_settings.kafka_topic_list.value; - if (args_count >= 2) - { - engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.local_context); - topic_list = engine_args[1]->as().value.safeGet(); - } - - Names topics; - boost::split(topics, topic_list , [](char c){ return c == ','; }); - for (String & topic : topics) - { - boost::trim(topic); - } - - // Get and check group name - String group = kafka_settings.kafka_group_name.value; - if (args_count >= 3) - { - engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context); - group = engine_args[2]->as().value.safeGet(); - } - - // Get and check message format name - String format = kafka_settings.kafka_format.value; - if (args_count >= 4) - { - engine_args[3] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[3], args.local_context); - - const auto * ast = engine_args[3]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - format = safeGet(ast->value); - } - else - { - throw Exception("Format must be a string", ErrorCodes::BAD_ARGUMENTS); - } - } - - // Parse row delimiter (optional) - char row_delimiter = kafka_settings.kafka_row_delimiter; - if (args_count >= 5) - { - engine_args[4] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[4], args.local_context); - - const auto * ast = engine_args[4]->as(); - String arg; - if (ast && ast->value.getType() == Field::Types::String) - { - arg = safeGet(ast->value); - } - else - { - throw Exception("Row delimiter must be a char", ErrorCodes::BAD_ARGUMENTS); - } - if (arg.size() > 1) - { - throw Exception("Row delimiter must be a char", ErrorCodes::BAD_ARGUMENTS); - } - else if (arg.empty()) - { - row_delimiter = '\0'; - } - else - { - row_delimiter = arg[0]; - } - } - - // Parse format schema if supported (optional) - String schema = kafka_settings.kafka_schema.value; - if (args_count >= 6) - { - engine_args[5] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[5], args.local_context); - - const auto * ast = engine_args[5]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - schema = safeGet(ast->value); - } - else - { - throw Exception("Format schema must be a string", ErrorCodes::BAD_ARGUMENTS); - } - } - - // Parse number of consumers (optional) - UInt64 num_consumers = kafka_settings.kafka_num_consumers; - if (args_count >= 7) - { - const auto * ast = engine_args[6]->as(); - if (ast && ast->value.getType() == Field::Types::UInt64) - { - num_consumers = safeGet(ast->value); - } - else - { - throw Exception("Number of consumers must be a positive integer", ErrorCodes::BAD_ARGUMENTS); - } - } - - // Parse max block size (optional) - UInt64 max_block_size = static_cast(kafka_settings.kafka_max_block_size); - if (args_count >= 8) - { - const auto * ast = engine_args[7]->as(); - if (ast && ast->value.getType() == Field::Types::UInt64) - { - max_block_size = static_cast(safeGet(ast->value)); - } - else - { - // TODO: no check if the integer is really positive - throw Exception("Maximum block size must be a positive integer", ErrorCodes::BAD_ARGUMENTS); - } - } - - size_t skip_broken = static_cast(kafka_settings.kafka_skip_broken_messages); - if (args_count >= 9) - { - const auto * ast = engine_args[8]->as(); - if (ast && ast->value.getType() == Field::Types::UInt64) - { - skip_broken = static_cast(safeGet(ast->value)); - } - else - { - throw Exception("Number of broken messages to skip must be a non-negative integer", ErrorCodes::BAD_ARGUMENTS); - } - } - - bool intermediate_commit = static_cast(kafka_settings.kafka_commit_every_batch); - if (args_count >= 10) - { - const auto * ast = engine_args[9]->as(); - if (ast && ast->value.getType() == Field::Types::UInt64) - { - intermediate_commit = static_cast(safeGet(ast->value)); - } - else - { - throw Exception("Flag for committing every batch must be 0 or 1", ErrorCodes::BAD_ARGUMENTS); - } - } - - // Get and check client id - String client_id = kafka_settings.kafka_client_id.value; - - return StorageKafka::create( - args.table_id, args.context, args.columns, - brokers, group, client_id, topics, format, row_delimiter, schema, num_consumers, max_block_size, skip_broken, intermediate_commit); + return StorageKafka::create(args.table_id, args.context, args.columns, std::move(kafka_settings)); }; factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 1ea7d6dcad7..be3f89687fe 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -54,10 +55,7 @@ public: ProducerBufferPtr createWriteBuffer(const Block & header); - const auto & getTopics() const { return topics; } const auto & getFormatName() const { return format_name; } - const auto & getSchemaName() const { return schema_name; } - const auto & skipBroken() const { return skip_broken; } NamesAndTypesList getVirtuals() const override; protected: @@ -65,58 +63,53 @@ protected: const StorageID & table_id_, Context & context_, const ColumnsDescription & columns_, - const String & brokers_, - const String & group_, - const String & client_id_, - const Names & topics_, - const String & format_name_, - char row_delimiter_, - const String & schema_name_, - size_t num_consumers_, - UInt64 max_block_size_, - size_t skip_broken, - bool intermediate_commit_); + std::unique_ptr kafka_settings_); private: // Configuration and state - Context global_context; - Context kafka_context; - Names topics; + Context & global_context; + std::shared_ptr kafka_context; + std::unique_ptr kafka_settings; + const Names topics; const String brokers; const String group; const String client_id; const String format_name; - char row_delimiter; /// optional row delimiter for generating char delimited stream in order to make various input stream parsers happy. + const char row_delimiter; /// optional row delimiter for generating char delimited stream in order to make various input stream parsers happy. const String schema_name; - size_t num_consumers; /// total number of consumers - UInt64 max_block_size; /// maximum block size for insertion into this table + const size_t num_consumers; /// total number of consumers + Poco::Logger * log; + Poco::Semaphore semaphore; + const bool intermediate_commit; + const SettingsChanges settings_adjustments; /// Can differ from num_consumers in case of exception in startup() (or if startup() hasn't been called). /// In this case we still need to be able to shutdown() properly. size_t num_created_consumers = 0; /// number of actually created consumers. - Poco::Logger * log; - - // Consumer list - Poco::Semaphore semaphore; - std::mutex mutex; std::vector buffers; /// available buffers for Kafka consumers - size_t skip_broken; - - bool intermediate_commit; + std::mutex mutex; // Stream thread BackgroundSchedulePool::TaskHolder task; std::atomic stream_cancelled{false}; + SettingsChanges createSettingsAdjustments(); ConsumerBufferPtr createReadBuffer(const size_t consumer_number); // Update Kafka configuration with values from CH user configuration. - void updateConfiguration(cppkafka::Configuration & conf); + void updateConfiguration(cppkafka::Configuration & conf); void threadFunc(); + + size_t getPollMaxBatchSize() const; + size_t getMaxBlockSize() const; + size_t getPollTimeoutMillisecond() const; + + static Names parseTopics(String topic_list); static String getDefaultClientId(const StorageID & table_id_); + bool streamToViews(); bool checkDependencies(const StorageID & table_id); }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 63d163a593e..fea05c00e4f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 26a02f8904c..e08ea1739a5 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -205,121 +205,32 @@ static void setGraphitePatternsFromConfig(const Context & context, } -static String getMergeTreeVerboseHelp(bool is_extended_syntax) +static String getMergeTreeVerboseHelp(bool) { using namespace std::string_literals; String help = R"( -MergeTree is a family of storage engines. +Syntax for the MergeTree table engine: -MergeTrees are different in two ways: -- they may be replicated and non-replicated; -- they may do different actions on merge: nothing; sign collapse; sum; apply aggregete functions. +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], + ... + INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, + INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2 +) ENGINE = MergeTree() +ORDER BY expr +[PARTITION BY expr] +[PRIMARY KEY expr] +[SAMPLE BY expr] +[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...] +[SETTINGS name=value, ...] -So we have 14 combinations: - MergeTree, CollapsingMergeTree, SummingMergeTree, AggregatingMergeTree, ReplacingMergeTree, GraphiteMergeTree, VersionedCollapsingMergeTree - ReplicatedMergeTree, ReplicatedCollapsingMergeTree, ReplicatedSummingMergeTree, ReplicatedAggregatingMergeTree, ReplicatedReplacingMergeTree, ReplicatedGraphiteMergeTree, ReplicatedVersionedCollapsingMergeTree +See details in documentation: https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/. Other engines of the family support different syntax, see details in the corresponding documentation topics. -In most of cases, you need MergeTree or ReplicatedMergeTree. - -For replicated merge trees, you need to supply a path in ZooKeeper and a replica name as the first two parameters. -Path in ZooKeeper is like '/clickhouse/tables/01/' where /clickhouse/tables/ is a common prefix and 01 is a shard name. -Replica name is like 'mtstat01-1' - it may be the hostname or any suitable string identifying replica. -You may use macro substitutions for these parameters. It's like ReplicatedMergeTree('/clickhouse/tables/{shard}/', '{replica}'... -Look at the section in server configuration file. -)"; - - if (!is_extended_syntax) - help += R"( -Next parameter (which is the first for unreplicated tables and the third for replicated tables) is the name of date column. -Date column must exist in the table and have type Date (not DateTime). -It is used for internal data partitioning and works like some kind of index. - -If your source data doesn't have a column of type Date, but has a DateTime column, you may add values for Date column while loading, - or you may INSERT your source data to a table of type Log and then transform it with INSERT INTO t SELECT toDate(time) AS date, * FROM ... -If your source data doesn't have any date or time, you may just pass any constant for a date column while loading. - -Next parameter is optional sampling expression. Sampling expression is used to implement SAMPLE clause in query for approximate query execution. -If you don't need approximate query execution, simply omit this parameter. -Sample expression must be one of the elements of the primary key tuple. For example, if your primary key is (CounterID, EventDate, intHash64(UserID)), your sampling expression might be intHash64(UserID). - -Next parameter is the primary key tuple. It's like (CounterID, EventDate, intHash64(UserID)) - a list of column names or functional expressions in round brackets. If your primary key has just one element, you may omit round brackets. - -Careful choice of the primary key is extremely important for processing short-time queries. - -Next parameter is index (primary key) granularity. Good value is 8192. You have no reasons to use any other value. -)"; - - help += R"( -For the Collapsing mode, the )" + (is_extended_syntax ? "only"s : "last"s) + R"( parameter is the name of a sign column - a special column that is used to 'collapse' rows with the same primary key while merging. - -For the Summing mode, the optional )" + (is_extended_syntax ? ""s : "last "s) + R"(parameter is a list of columns to sum while merging. This list is passed in round brackets, like (PageViews, Cost). -If this parameter is omitted, the storage will sum all numeric columns except columns participating in the primary key. - -For the Replacing mode, the optional )" + (is_extended_syntax ? ""s : "last "s) + R"(parameter is the name of a 'version' column. While merging, for all rows with the same primary key, only one row is selected: the last row, if the version column was not specified, or the last row with the maximum version value, if specified. - -For VersionedCollapsing mode, the )" + (is_extended_syntax ? ""s : "last "s) + R"(2 parameters are the name of a sign column and the name of a 'version' column. Version column must be in primary key. While merging, a pair of rows with the same primary key and different sign may collapse. -)"; - - if (is_extended_syntax) - help += R"( -You can specify a partitioning expression in the PARTITION BY clause. It is optional but highly recommended. -A common partitioning expression is some function of the event date column e.g. PARTITION BY toYYYYMM(EventDate) will partition the table by month. -Rows with different partition expression values are never merged together. That allows manipulating partitions with ALTER commands. -Also it acts as a kind of index. - -Sorting key is specified in the ORDER BY clause. It is mandatory for all MergeTree types. -It is like (CounterID, EventDate, intHash64(UserID)) - a list of column names or functional expressions -in round brackets. -If your sorting key has just one element, you may omit round brackets. - -By default primary key is equal to the sorting key. You can specify a primary key that is a prefix of the -sorting key in the PRIMARY KEY clause. - -Careful choice of the primary key is extremely important for processing short-time queries. - -Optional sampling expression can be specified in the SAMPLE BY clause. It is used to implement the SAMPLE clause in a SELECT query for approximate query execution. -Sampling expression must be one of the elements of the primary key tuple. For example, if your primary key is (CounterID, EventDate, intHash64(UserID)), your sampling expression might be intHash64(UserID). - -Engine settings can be specified in the SETTINGS clause. Full list is in the source code in the 'src/Storages/MergeTree/MergeTreeSettings.h' file. -E.g. you can specify the index (primary key) granularity with SETTINGS index_granularity = 8192. - -Examples: - -MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate) SETTINGS index_granularity = 8192 - -MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) SAMPLE BY intHash32(UserID) - -MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) PRIMARY KEY (CounterID, EventDate) SAMPLE BY intHash32(UserID) - -CollapsingMergeTree(Sign) PARTITION BY StartDate SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) - -SummingMergeTree PARTITION BY toMonday(EventDate) ORDER BY (OrderID, EventDate, BannerID, PhraseID, ContextType, RegionID, PageID, IsFlat, TypeID, ResourceNo) - -SummingMergeTree((Shows, Clicks, Cost, CostCur, ShowsSumPosition, ClicksSumPosition, SessionNum, SessionLen, SessionCost, GoalsNum, SessionDepth)) PARTITION BY toYYYYMM(EventDate) ORDER BY (OrderID, EventDate, BannerID, PhraseID, ContextType, RegionID, PageID, IsFlat, TypeID, ResourceNo) - -ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}') PARTITION BY EventDate ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) SAMPLE BY intHash32(UserID) -)"; - else - help += R"( -Examples: - -MergeTree(EventDate, (CounterID, EventDate), 8192) - -MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) - -CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign) - -SummingMergeTree(EventDate, (OrderID, EventDate, BannerID, PhraseID, ContextType, RegionID, PageID, IsFlat, TypeID, ResourceNo), 8192) - -SummingMergeTree(EventDate, (OrderID, EventDate, BannerID, PhraseID, ContextType, RegionID, PageID, IsFlat, TypeID, ResourceNo), 8192, (Shows, Clicks, Cost, CostCur, ShowsSumPosition, ClicksSumPosition, SessionNum, SessionLen, SessionCost, GoalsNum, SessionDepth)) - -ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) -)"; - - help += R"( -For further info please read the documentation: https://clickhouse.yandex/ +If you use the Replicated version of engines, see https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/replication/. )"; return help; diff --git a/src/Storages/StorageHDFS.cpp b/src/Storages/StorageHDFS.cpp index 562ea5c9486..352e0a43f39 100644 --- a/src/Storages/StorageHDFS.cpp +++ b/src/Storages/StorageHDFS.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -28,6 +27,7 @@ #include #include + namespace DB { namespace ErrorCodes diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 918662750e4..397d064ba15 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -35,6 +35,8 @@ #include #include +#include + namespace DB { @@ -249,11 +251,17 @@ Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI & outcome = client.ListObjectsV2(request); if (!outcome.IsSuccess()) { - throw Exception("Could not list objects in bucket " + quoteString(request.GetBucket()) - + " with prefix " + quoteString(request.GetPrefix()) - + ", page " + std::to_string(page) - + ", S3 exception " + outcome.GetError().GetExceptionName() + " " + outcome.GetError().GetMessage() - , ErrorCodes::S3_ERROR); + std::ostringstream message; + message << "Could not list objects in bucket " << quoteString(request.GetBucket()) + << " with prefix " << quoteString(request.GetPrefix()); + + if (page > 1) + message << ", page " << std::to_string(page); + + message << ", S3 exception: " + backQuote(outcome.GetError().GetExceptionName()) + << ", message: " + quoteString(outcome.GetError().GetMessage()); + + throw Exception(message.str(), ErrorCodes::S3_ERROR); } for (const auto & row : outcome.GetResult().GetContents()) diff --git a/src/Storages/tests/CMakeLists.txt b/src/Storages/tests/CMakeLists.txt index 80dd4c8419c..a6ad49e562e 100644 --- a/src/Storages/tests/CMakeLists.txt +++ b/src/Storages/tests/CMakeLists.txt @@ -1,9 +1,3 @@ -add_executable (system_numbers system_numbers.cpp) -target_link_libraries (system_numbers PRIVATE dbms clickhouse_storages_system clickhouse_common_io) - -add_executable (storage_log storage_log.cpp) -target_link_libraries (storage_log PRIVATE dbms) - add_executable (part_name part_name.cpp) target_link_libraries (part_name PRIVATE dbms) diff --git a/src/Storages/tests/storage_log.cpp b/src/Storages/tests/storage_log.cpp deleted file mode 100644 index 1e1ab4c682f..00000000000 --- a/src/Storages/tests/storage_log.cpp +++ /dev/null @@ -1,113 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -int main(int, char **) -try -{ - using namespace DB; - - const size_t rows = 10000000; - - /// create table with a pair of columns - - NamesAndTypesList names_and_types; - names_and_types.emplace_back("a", std::make_shared()); - names_and_types.emplace_back("b", std::make_shared()); - - SharedContextHolder shared_context = Context::createShared(); - auto context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); - context.setPath("./"); - - DiskPtr disk = std::make_unique("default", "./", 0); - StoragePtr table = StorageLog::create(disk, "table/", StorageID("test", "test"), ColumnsDescription{names_and_types}, ConstraintsDescription{}, 1048576); - - table->startup(); - - /// write into it - { - Block block; - - { - ColumnWithTypeAndName column; - column.name = "a"; - column.type = table->getColumns().getPhysical("a").type; - auto col = column.type->createColumn(); - ColumnUInt64::Container & vec = typeid_cast(*col).getData(); - - vec.resize(rows); - for (size_t i = 0; i < rows; ++i) - vec[i] = i; - - column.column = std::move(col); - block.insert(column); - } - - { - ColumnWithTypeAndName column; - column.name = "b"; - column.type = table->getColumns().getPhysical("b").type; - auto col = column.type->createColumn(); - ColumnUInt8::Container & vec = typeid_cast(*col).getData(); - - vec.resize(rows); - for (size_t i = 0; i < rows; ++i) - vec[i] = i * 2; - - column.column = std::move(col); - block.insert(column); - } - - BlockOutputStreamPtr out = table->write({}, context); - out->write(block); - } - - /// read from it - { - Names column_names; - column_names.push_back("a"); - column_names.push_back("b"); - - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); - - BlockInputStreamPtr in = std::make_shared(std::move(table->read(column_names, {}, context, stage, 8192, 1)[0])); - - Block sample; - { - ColumnWithTypeAndName col; - col.type = std::make_shared(); - sample.insert(std::move(col)); - } - { - ColumnWithTypeAndName col; - col.type = std::make_shared(); - sample.insert(std::move(col)); - } - - WriteBufferFromOStream out_buf(std::cout); - - LimitBlockInputStream in_limit(in, 10, 0); - BlockOutputStreamPtr output = FormatFactory::instance().getOutput("TabSeparated", out_buf, sample, context); - - copyData(in_limit, *output); - } - - return 0; -} -catch (const DB::Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - return 1; -} diff --git a/src/Storages/tests/system_numbers.cpp b/src/Storages/tests/system_numbers.cpp deleted file mode 100644 index 6955c90b74e..00000000000 --- a/src/Storages/tests/system_numbers.cpp +++ /dev/null @@ -1,47 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include - - -int main(int, char **) -try -{ - using namespace DB; - - StoragePtr table = StorageSystemNumbers::create(StorageID("test", "numbers"), false); - - Names column_names; - column_names.push_back("number"); - - Block sample; - ColumnWithTypeAndName col; - col.type = std::make_shared(); - sample.insert(std::move(col)); - - WriteBufferFromOStream out_buf(std::cout); - - SharedContextHolder shared_context = Context::createShared(); - auto context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); - - auto stream = std::make_shared(std::move(table->read(column_names, {}, context, stage, 10, 1)[0])); - LimitBlockInputStream input(stream, 10, 96); - BlockOutputStreamPtr out = FormatFactory::instance().getOutput("TabSeparated", out_buf, sample, context); - - copyData(input, *out); - - return 0; -} -catch (const DB::Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - return 1; -} diff --git a/src/Storages/tests/test_alter_distributed.sql b/src/Storages/tests/test_alter_distributed.sql deleted file mode 100644 index 0578d340276..00000000000 --- a/src/Storages/tests/test_alter_distributed.sql +++ /dev/null @@ -1,28 +0,0 @@ -create database if not exists test; - -drop table if exists test.merge_distributed; -drop table if exists test.merge_distributed1; - -create table test.merge_distributed1 ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); -insert into test.merge_distributed1 values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); - -create table test.merge_distributed ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = Distributed(self, test, merge_distributed1); - -alter table test.merge_distributed1 add column dummy String after CounterID; -alter table test.merge_distributed add column dummy String after CounterID; - -describe table test.merge_distributed; -show create table test.merge_distributed; - -insert into test.merge_distributed1 values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); -select CounterID, dummy from test.merge_distributed where dummy <> '' limit 10; - -alter table test.merge_distributed drop column dummy; - -describe table test.merge_distributed; -show create table test.merge_distributed; - ---error: should fall, because there is no `dummy1` column -alter table test.merge_distributed add column dummy1 String after CounterID; -select CounterID, dummy1 from test.merge_distributed where dummy1 <> '' limit 10; - diff --git a/src/Storages/tests/test_alter_merge.sql b/src/Storages/tests/test_alter_merge.sql deleted file mode 100644 index 252577ddc37..00000000000 --- a/src/Storages/tests/test_alter_merge.sql +++ /dev/null @@ -1,35 +0,0 @@ -create database if not exists test; - -drop table if exists test.merge; -drop table if exists test.merge1; -drop table if exists test.merge2; - -create table test.merge1 ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); -insert into test.merge1 values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); - -create table test.merge2 ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); -insert into test.merge2 values (2, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); - -create table test.merge ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = Merge(test, 'merge\[0-9\]'); - -alter table test.merge1 add column dummy String after CounterID; -alter table test.merge2 add column dummy String after CounterID; -alter table test.merge add column dummy String after CounterID; - -describe table test.merge; -show create table test.merge; - -insert into test.merge1 values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); - -select CounterID, dummy from test.merge where dummy <> '' limit 10; - - -alter table test.merge drop column dummy; - -describe table test.merge; -show create table test.merge; - ---error: must correctly fall into the alter -alter table test.merge add column dummy1 String after CounterID; -select CounterID, dummy1 from test.merge where dummy1 <> '' limit 10; - diff --git a/src/Storages/tests/test_alter_merge_tree.sql b/src/Storages/tests/test_alter_merge_tree.sql deleted file mode 100644 index 5ac361acc5c..00000000000 --- a/src/Storages/tests/test_alter_merge_tree.sql +++ /dev/null @@ -1,17 +0,0 @@ -create database if not exists test; - -drop table if exists test.merge_tree; - -create table test.merge_tree ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); - -insert into test.merge_tree values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3) -alter table test.merge_tree add column dummy String after CounterID; -describe table test.merge_tree; - -insert into test.merge_tree values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3) - -select CounterID, dummy from test.merge_tree where dummy <> '' limit 10; - -alter table test.merge_tree drop column dummy; - -describe table test.merge_tree; diff --git a/tests/integration/test_host_ip_change/test.py b/tests/integration/test_host_ip_change/test.py index 1b6d4365ac9..ac35478277c 100644 --- a/tests/integration/test_host_ip_change/test.py +++ b/tests/integration/test_host_ip_change/test.py @@ -76,8 +76,6 @@ node3 = cluster.add_instance('node3', main_configs=['configs/listen_host.xml'], with_zookeeper=True, ipv6_address='2001:3984:3989::1:1113') node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/listen_host.xml', 'configs/dns_update_short.xml'], with_zookeeper=True, ipv6_address='2001:3984:3989::1:1114') -node5 = cluster.add_instance('node5', main_configs=['configs/listen_host.xml', 'configs/dns_update_short.xml'], - user_configs=['configs/users_with_hostname.xml'], ipv6_address='2001:3984:3989::1:1115') @pytest.fixture(scope="module") def cluster_with_dns_cache_update(): @@ -142,24 +140,39 @@ def test_dns_cache_update(cluster_with_dns_cache_update): assert TSV(node4.query("SELECT DISTINCT host_name, host_address FROM system.clusters WHERE cluster='lost_host_cluster'")) == TSV("lost_host\t127.0.0.1\n") assert TSV(node4.query("SELECT hostName()")) == TSV("node4") -def test_user_access_ip_change(cluster_with_dns_cache_update): - assert node3.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n" - assert node4.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n" +# Check SYSTEM DROP DNS CACHE on node5 and background cache update on node6 +node5 = cluster.add_instance('node5', main_configs=['configs/listen_host.xml', 'configs/dns_update_long.xml'], + user_configs=['configs/users_with_hostname.xml'], ipv6_address='2001:3984:3989::1:1115') +node6 = cluster.add_instance('node6', main_configs=['configs/listen_host.xml', 'configs/dns_update_short.xml'], + user_configs=['configs/users_with_hostname.xml'], ipv6_address='2001:3984:3989::1:1116') - set_hosts(node5, ['127.255.255.255 node3', '2001:3984:3989::1:8884 unknown_host']) +@pytest.mark.parametrize("node", [node5, node6]) +def test_user_access_ip_change(cluster_with_dns_cache_update, node): + node_name = node.name + node_num = node.name[-1] + # getaddrinfo(...) may hang for a log time without this options + node.exec_in_container(['bash', '-c', 'echo -e "options timeout:1\noptions attempts:2" >> /etc/resolv.conf'], privileged=True, user='root') - cluster.restart_instance_with_ip_change(node3, "2001:3984:3989::1:8883") - cluster.restart_instance_with_ip_change(node4, "2001:3984:3989::1:8884") + assert node3.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name)) == "0\n" + assert node4.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name)) == "0\n" + + set_hosts(node, ['127.255.255.255 node3', '2001:3984:3989::1:88{}4 unknown_host'.format(node_num)]) + + cluster.restart_instance_with_ip_change(node3, "2001:3984:3989::1:88{}3".format(node_num)) + cluster.restart_instance_with_ip_change(node4, "2001:3984:3989::1:88{}4".format(node_num)) with pytest.raises(QueryRuntimeException): - node3.query("SELECT * FROM remote('node5', 'system', 'one')") + node3.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name)) with pytest.raises(QueryRuntimeException): - node4.query("SELECT * FROM remote('node5', 'system', 'one')") + node4.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name)) # now wrong addresses are cached - set_hosts(node5, []) - # client is not allowed to connect, so execute it directly in container to send query from localhost - node5.exec_in_container(['bash', '-c', 'clickhouse client -q "SYSTEM DROP DNS CACHE"'], privileged=True, user='root') + set_hosts(node, []) + retry_count = 60 + if node_name == 'node5': + # client is not allowed to connect, so execute it directly in container to send query from localhost + node.exec_in_container(['bash', '-c', 'clickhouse client -q "SYSTEM DROP DNS CACHE"'], privileged=True, user='root') + retry_count = 1 - assert node3.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n" - assert node4.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n" + assert_eq_with_retry(node3, "SELECT * FROM remote('{}', 'system', 'one')".format(node_name), "0", retry_count=retry_count, sleep_time=1) + assert_eq_with_retry(node4, "SELECT * FROM remote('{}', 'system', 'one')".format(node_name), "0", retry_count=retry_count, sleep_time=1) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index d2a2532bb9a..2a1b42f8e0e 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -233,6 +233,83 @@ def test_kafka_settings_new_syntax(kafka_cluster): members = describe_consumer_group('new') assert members[0]['client_id'] == u'instance test 1234' + +@pytest.mark.timeout(180) +def test_kafka_issue11308(kafka_cluster): + # Check that matview does respect Kafka SETTINGS + kafka_produce('issue11308', ['{"t": 123, "e": {"x": "woof"} }', '{"t": 123, "e": {"x": "woof"} }', '{"t": 124, "e": {"x": "test"} }']) + + instance.query(''' + CREATE TABLE test.persistent_kafka ( + time UInt64, + some_string String + ) + ENGINE = MergeTree() + ORDER BY time; + + CREATE TABLE test.kafka (t UInt64, `e.x` String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'issue11308', + kafka_group_name = 'issue11308', + kafka_format = 'JSONEachRow', + kafka_row_delimiter = '\\n', + kafka_flush_interval_ms=1000, + input_format_import_nested_json = 1; + + CREATE MATERIALIZED VIEW test.persistent_kafka_mv TO test.persistent_kafka AS + SELECT + `t` AS `time`, + `e.x` AS `some_string` + FROM test.kafka; + ''') + + time.sleep(9) + + result = instance.query('SELECT * FROM test.persistent_kafka ORDER BY time;') + + instance.query(''' + DROP TABLE test.persistent_kafka; + DROP TABLE test.persistent_kafka_mv; + ''') + + expected = '''\ +123 woof +123 woof +124 test +''' + assert TSV(result) == TSV(expected) + + +@pytest.mark.timeout(180) +def test_kafka_issue4116(kafka_cluster): + # Check that format_csv_delimiter parameter works now - as part of all available format settings. + kafka_produce('issue4116', ['1|foo', '2|bar', '42|answer','100|multi\n101|row\n103|message']) + + instance.query(''' + CREATE TABLE test.kafka (a UInt64, b String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'issue4116', + kafka_group_name = 'issue4116', + kafka_format = 'CSV', + kafka_row_delimiter = '\\n', + format_csv_delimiter = '|'; + ''') + + result = instance.query('SELECT * FROM test.kafka ORDER BY a;') + + expected = '''\ +1 foo +2 bar +42 answer +100 multi +101 row +103 message +''' + assert TSV(result) == TSV(expected) + + @pytest.mark.timeout(180) def test_kafka_consumer_hang(kafka_cluster): diff --git a/tests/queries/0_stateless/01297_alter_distributed.reference b/tests/queries/0_stateless/01297_alter_distributed.reference new file mode 100644 index 00000000000..bd269322884 --- /dev/null +++ b/tests/queries/0_stateless/01297_alter_distributed.reference @@ -0,0 +1,18 @@ +CounterID UInt32 +dummy String +StartDate Date +Sign Int8 +VisitID UInt64 +UserID UInt64 +StartTime DateTime +ClickLogID UInt64 +CREATE TABLE default.merge_distributed\n(\n `CounterID` UInt32, \n `dummy` String, \n `StartDate` Date, \n `Sign` Int8, \n `VisitID` UInt64, \n `UserID` UInt64, \n `StartTime` DateTime, \n `ClickLogID` UInt64\n)\nENGINE = Distributed(\'test_shard_localhost\', \'default\', \'merge_distributed1\') +1 Hello, Alter Table! +CounterID UInt32 +StartDate Date +Sign Int8 +VisitID UInt64 +UserID UInt64 +StartTime DateTime +ClickLogID UInt64 +CREATE TABLE default.merge_distributed\n(\n `CounterID` UInt32, \n `StartDate` Date, \n `Sign` Int8, \n `VisitID` UInt64, \n `UserID` UInt64, \n `StartTime` DateTime, \n `ClickLogID` UInt64\n)\nENGINE = Distributed(\'test_shard_localhost\', \'default\', \'merge_distributed1\') diff --git a/tests/queries/0_stateless/01297_alter_distributed.sql b/tests/queries/0_stateless/01297_alter_distributed.sql new file mode 100644 index 00000000000..d5359cc5ea8 --- /dev/null +++ b/tests/queries/0_stateless/01297_alter_distributed.sql @@ -0,0 +1,28 @@ +drop table if exists merge_distributed; +drop table if exists merge_distributed1; + +create table merge_distributed1 ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); +insert into merge_distributed1 values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); + +create table merge_distributed ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = Distributed(test_shard_localhost, currentDatabase(), merge_distributed1); + +alter table merge_distributed1 add column dummy String after CounterID; +alter table merge_distributed add column dummy String after CounterID; + +describe table merge_distributed; +show create table merge_distributed; + +insert into merge_distributed1 values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); +select CounterID, dummy from merge_distributed where dummy <> '' limit 10; + +alter table merge_distributed drop column dummy; + +describe table merge_distributed; +show create table merge_distributed; + +--error: should fall, because there is no `dummy1` column +alter table merge_distributed add column dummy1 String after CounterID; +select CounterID, dummy1 from merge_distributed where dummy1 <> '' limit 10; -- { serverError 47 } + +drop table merge_distributed; +drop table merge_distributed1; diff --git a/tests/queries/0_stateless/01298_alter_merge.reference b/tests/queries/0_stateless/01298_alter_merge.reference new file mode 100644 index 00000000000..393c0a600ff --- /dev/null +++ b/tests/queries/0_stateless/01298_alter_merge.reference @@ -0,0 +1,17 @@ +CounterID UInt32 +dummy String +StartDate Date +Sign Int8 +VisitID UInt64 +UserID UInt64 +StartTime DateTime +ClickLogID UInt64 +CREATE TABLE default.merge\n(\n `CounterID` UInt32, \n `dummy` String, \n `StartDate` Date, \n `Sign` Int8, \n `VisitID` UInt64, \n `UserID` UInt64, \n `StartTime` DateTime, \n `ClickLogID` UInt64\n)\nENGINE = Merge(\'default\', \'merge\\\\[0-9\\\\]\') +CounterID UInt32 +StartDate Date +Sign Int8 +VisitID UInt64 +UserID UInt64 +StartTime DateTime +ClickLogID UInt64 +CREATE TABLE default.merge\n(\n `CounterID` UInt32, \n `StartDate` Date, \n `Sign` Int8, \n `VisitID` UInt64, \n `UserID` UInt64, \n `StartTime` DateTime, \n `ClickLogID` UInt64\n)\nENGINE = Merge(\'default\', \'merge\\\\[0-9\\\\]\') diff --git a/tests/queries/0_stateless/01298_alter_merge.sql b/tests/queries/0_stateless/01298_alter_merge.sql new file mode 100644 index 00000000000..86c89c38c8c --- /dev/null +++ b/tests/queries/0_stateless/01298_alter_merge.sql @@ -0,0 +1,36 @@ +drop table if exists merge; +drop table if exists merge1; +drop table if exists merge2; + +create table merge1 ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); +insert into merge1 values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); + +create table merge2 ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); +insert into merge2 values (2, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); + +create table merge ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = Merge(currentDatabase(), 'merge\[0-9\]'); + +alter table merge1 add column dummy String after CounterID; +alter table merge2 add column dummy String after CounterID; +alter table merge add column dummy String after CounterID; + +describe table merge; +show create table merge; + +insert into merge1 values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3); + +select CounterID, dummy from merge where dummy <> '' limit 10; + + +alter table merge drop column dummy; + +describe table merge; +show create table merge; + +--error: must correctly fall into the alter +alter table merge add column dummy1 String after CounterID; +select CounterID, dummy1 from merge where dummy1 <> '' limit 10; + +drop table merge; +drop table merge1; +drop table merge2; diff --git a/tests/queries/0_stateless/01299_alter_merge_tree.reference b/tests/queries/0_stateless/01299_alter_merge_tree.reference new file mode 100644 index 00000000000..d641115026b --- /dev/null +++ b/tests/queries/0_stateless/01299_alter_merge_tree.reference @@ -0,0 +1,16 @@ +CounterID UInt32 +dummy String +StartDate Date +Sign Int8 +VisitID UInt64 +UserID UInt64 +StartTime DateTime +ClickLogID UInt64 +1 Hello, Alter Table! +CounterID UInt32 +StartDate Date +Sign Int8 +VisitID UInt64 +UserID UInt64 +StartTime DateTime +ClickLogID UInt64 diff --git a/tests/queries/0_stateless/01299_alter_merge_tree.sql b/tests/queries/0_stateless/01299_alter_merge_tree.sql new file mode 100644 index 00000000000..87608e6d15a --- /dev/null +++ b/tests/queries/0_stateless/01299_alter_merge_tree.sql @@ -0,0 +1,17 @@ +drop table if exists merge_tree; + +create table merge_tree ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign); + +insert into merge_tree values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3) +alter table merge_tree add column dummy String after CounterID; +describe table merge_tree; + +insert into merge_tree values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3) + +select CounterID, dummy from merge_tree where dummy <> '' limit 10; + +alter table merge_tree drop column dummy; + +describe table merge_tree; + +drop table merge_tree; diff --git a/tests/queries/0_stateless/01300_client_save_history_when_terminated.reference b/tests/queries/0_stateless/01300_client_save_history_when_terminated.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01300_client_save_history_when_terminated.sh b/tests/queries/0_stateless/01300_client_save_history_when_terminated.sh new file mode 100755 index 00000000000..5ffcbbda883 --- /dev/null +++ b/tests/queries/0_stateless/01300_client_save_history_when_terminated.sh @@ -0,0 +1,35 @@ +#!/usr/bin/expect -f + +log_user 0 +set timeout 60 +match_max 100000 + +spawn clickhouse-client +expect ":) " + +# Make a query +send -- "SELECT 'for the history'\r" +expect "for the history" +expect ":) " + +# Kill the client to check if the history was saved +exec kill -9 [exp_pid] +close + +# Run client one more time and press "up" to see the last recorded query +spawn clickhouse-client +expect ":) " +send -- "\[A" +expect "SELECT 'for the history'" + +# Will check that Ctrl+C clears current line. +send -- "\3" +expect ":)" + +# Will check that second Ctrl+C invocation does not exit from client. +send -- "\3" +expect ":)" + +# But Ctrl+D does. +send -- "\4" +expect eof