Merge branch 'master' into kuskarov-add_syntax_highlighting

This commit is contained in:
alexey-milovidov 2020-06-08 02:14:09 +03:00 committed by GitHub
commit ce4f528b73
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
74 changed files with 2241 additions and 3468 deletions

View File

@ -16,6 +16,7 @@ set (SRCS
shift10.cpp
sleep.cpp
terminalColors.cpp
errnoToString.cpp
)
if (ENABLE_REPLXX)

View File

@ -1,9 +1,11 @@
#include <common/ReplxxLineReader.h>
#include <common/errnoToString.h>
#include <errno.h>
#include <string.h>
#include <unistd.h>
#include <functional>
#include <sys/file.h>
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()

View File

@ -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;
};

View File

@ -0,0 +1,29 @@
#include "errnoToString.h"
#include <fmt/format.h>
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
}

View File

@ -0,0 +1,6 @@
#pragma once
#include <cerrno>
#include <string>
std::string errnoToString(int code, int the_errno = errno);

View File

@ -47,6 +47,7 @@ SRCS(
shift10.cpp
sleep.cpp
terminalColors.cpp
errnoToString.cpp
)
END()

View File

@ -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)

View File

@ -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

View File

@ -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.

View File

@ -17,6 +17,7 @@
#include <common/phdr_cache.h>
#include <common/ErrorHandlers.h>
#include <common/getMemoryAmount.h>
#include <common/errnoToString.h>
#include <common/coverage.h>
#include <Common/ClickHouseRevision.h>
#include <Common/DNSResolver.h>

View File

@ -8,6 +8,7 @@
#include <ext/scope_guard.h>
#include <boost/algorithm/string/replace.hpp>
#include <ifaddrs.h>
#include <Common/DNSResolver.h>
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<const sockaddr_in *>(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<const sockaddr_in6*>(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<decltype(isAddressOfHostImpl), isAddressOfHostImpl> 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<decltype(getHostByAddressImpl), &getHostByAddressImpl> 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();
}
}

View File

@ -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<IPAddress> addresses;
std::vector<IPSubnet> subnets;

View File

@ -4,20 +4,32 @@
#include <Common/ProfileEvents.h>
#include <Core/Names.h>
#include <Core/Types.h>
#include <Poco/Net/IPAddress.h>
#include <Poco/Net/DNS.h>
#include <Poco/Net/NetException.h>
#include <Poco/NumberParser.h>
#include <Poco/Logger.h>
#include <common/logger_useful.h>
#include <arpa/inet.h>
#include <atomic>
#include <optional>
#include <string_view>
namespace ProfileEvents
{
extern Event DNSError;
}
namespace std
{
template<> struct hash<Poco::Net::IPAddress>
{
size_t operator()(const Poco::Net::IPAddress & address) const noexcept
{
std::string_view addr(static_cast<const char *>(address.addr()), address.length());
std::hash<std::string_view> 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<decltype(resolveIPAddressImpl), &resolveIPAddressImpl> cache_host;
SimpleCache<decltype(reverseResolveImpl), &reverseResolveImpl> 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<Poco::Net::IPAddress> new_addresses;
/// Store all hosts, which was whenever asked to resolve
NameSet known_hosts;
std::unordered_set<Poco::Net::IPAddress> known_addresses;
/// If disabled, will not make cache lookups, will resolve addresses manually on each call
std::atomic<bool> disable_cache{false};
};
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()) {}
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()), 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<typename UpdateF, typename ElemsT>
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()

View File

@ -5,6 +5,7 @@
#include <Core/Types.h>
#include <Core/Names.h>
#include <boost/noncopyable.hpp>
#include <common/logger_useful.h>
namespace DB
@ -16,18 +17,26 @@ namespace DB
class DNSResolver : private boost::noncopyable
{
public:
typedef std::vector<Poco::Net::IPAddress> 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<typename UpdateF, typename ElemsT>
bool updateCacheImpl(UpdateF && update_func, ElemsT && elems, const String & log_msg);
DNSResolver();
struct Impl;
std::unique_ptr<Impl> 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);
};
}

View File

@ -10,6 +10,7 @@
#include <IO/ReadBufferFromString.h>
#include <IO/ReadBufferFromFile.h>
#include <common/demangle.h>
#include <common/errnoToString.h>
#include <Common/formatReadable.h>
#include <Common/filesystemHelpers.h>
#include <filesystem>
@ -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);

View File

@ -81,7 +81,6 @@ private:
using Exceptions = std::vector<std::exception_ptr>;
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,

View File

@ -3,6 +3,7 @@
#include <Common/formatReadable.h>
#include <common/logger_useful.h>
#include <common/errnoToString.h>
#include <unistd.h>
#include <fcntl.h>

View File

@ -8,6 +8,7 @@
#include <common/StringRef.h>
#include <common/logger_useful.h>
#include <common/phdr_cache.h>
#include <common/errnoToString.h>
#include <random>

View File

@ -6,6 +6,7 @@
#include <Common/ShellCommand.h>
#include <Common/PipeFDs.h>
#include <common/logger_useful.h>
#include <common/errnoToString.h>
#include <IO/WriteHelpers.h>
#include <unistd.h>
#include <csignal>

View File

@ -8,6 +8,7 @@
#include <Poco/File.h>
#include <common/logger_useful.h>
#include <common/errnoToString.h>
#include <Common/ClickHouseRevision.h>
#include <common/LocalDateTime.h>

View File

@ -238,7 +238,6 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
{
static_assert(is_unsigned_v<ValueType>, "ValueType must be unsigned.");
using UnsignedDeltaType = ValueType;
using SignedDeltaType = typename std::make_signed<UnsignedDeltaType>::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<SignedDeltaType>(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<UnsignedDeltaType>(signed_dd);
}
const UnsignedDeltaType delta = double_delta + prev_delta;

View File

@ -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);

File diff suppressed because it is too large Load Diff

View File

@ -46,7 +46,7 @@ struct Settings : public SettingsCollection<Settings>
* 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<Settings>
\
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<Settings>
\
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<Settings>
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<Settings>
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<Settings>
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)

View File

@ -1,4 +1,3 @@
#include <DataStreams/ExpressionBlockInputStream.h>
#include <DataStreams/CheckConstraintsBlockOutputStream.h>
#include <Parsers/formatAST.h>
#include <Interpreters/ExpressionActions.h>

View File

@ -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)

View File

@ -1,86 +0,0 @@
#include <iostream>
#include <iomanip>
#include <IO/WriteBufferFromOStream.h>
#include <IO/ReadHelpers.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <DataStreams/LimitBlockInputStream.h>
#include <DataStreams/ExpressionBlockInputStream.h>
#include <Formats/FormatFactory.h>
#include <DataStreams/copyData.h>
#include <DataTypes/DataTypesNumber.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/Context.h>
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
int main(int argc, char ** argv)
try
{
using namespace DB;
size_t n = argc == 2 ? parse<UInt64>(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<DataTypeUInt64>()}};
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<TreeExecutorBlockInputStream>(std::move(table->read(column_names, {}, context, stage, 8192, 1)[0]));
in = std::make_shared<ExpressionBlockInputStream>(in, expression);
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(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;
}

View File

@ -1,89 +0,0 @@
#include <iostream>
#include <iomanip>
#include <IO/WriteBufferFromOStream.h>
#include <IO/ReadHelpers.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <DataStreams/LimitBlockInputStream.h>
#include <DataStreams/ExpressionBlockInputStream.h>
#include <DataStreams/FilterBlockInputStream.h>
#include <Formats/FormatFactory.h>
#include <DataStreams/copyData.h>
#include <DataTypes/DataTypesNumber.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/Context.h>
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
int main(int argc, char ** argv)
try
{
using namespace DB;
size_t n = argc == 2 ? parse<UInt64>(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<DataTypeUInt64>()}};
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<TreeExecutorBlockInputStream>(std::move(table->read(column_names, {}, context, stage, 8192, 1)[0]));
in = std::make_shared<FilterBlockInputStream>(in, expression, "equals(modulo(number, 3), 1)");
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(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;
}

View File

@ -1,66 +0,0 @@
#include <iostream>
#include <iomanip>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <DataStreams/LimitBlockInputStream.h>
#include <DataStreams/UnionBlockInputStream.h>
#include <DataStreams/AsynchronousBlockInputStream.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/copyData.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Interpreters/loadMetadata.h>
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
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<AsynchronousBlockInputStream>(std::make_shared<TreeExecutorBlockInputStream>(std::move(pipes[i])));
BlockInputStreamPtr stream = std::make_shared<UnionBlockInputStream>(streams, nullptr, settings.max_threads);
stream = std::make_shared<LimitBlockInputStream>(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;
}

View File

@ -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);
}
}

View File

@ -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)

View File

@ -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,

View File

@ -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;

View File

@ -8,6 +8,7 @@
#include <Common/QueryProfiler.h>
#include <Common/ThreadProfileEvents.h>
#include <Common/TraceCollector.h>
#include <common/errnoToString.h>
#if defined(OS_LINUX)
# include <Common/hasLinuxCapability.h>

View File

@ -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)

View File

@ -1,105 +0,0 @@
#include <iostream>
#include <iomanip>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/Aggregator.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
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<DataTypeInt16>();
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<DataTypeString>();
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<DataTypeString>();
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<OneBlockInputStream>(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;
}

View File

@ -1,103 +0,0 @@
#include <iostream>
#include <iomanip>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Databases/DatabaseOrdinary.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
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<DatabaseOrdinary>("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;
}

View File

@ -1,140 +0,0 @@
#include <iostream>
#include <iomanip>
#include <IO/WriteBufferFromOStream.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Formats/FormatFactory.h>
#include <DataStreams/LimitBlockInputStream.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataStreams/copyData.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/Context.h>
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<DataTypeInt16>()},
{"s1", std::make_shared<DataTypeString>()},
{"s2", std::make_shared<DataTypeString>()}
};
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<DataTypeInt16>();
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<DataTypeString>();
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<DataTypeString>();
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<OneBlockInputStream>(block);
LimitBlockInputStream lis(is, 20, std::max(0, static_cast<int>(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;
}

View File

@ -1,61 +0,0 @@
#include <iostream>
#include <iomanip>
#include <common/DateLUT.h>
#include <Poco/ConsoleChannel.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <Storages/StorageLog.h>
#include <Storages/System/attachSystemTables.h>
#include <Interpreters/Context.h>
#include <Interpreters/loadMetadata.h>
#include <Interpreters/executeQuery.h>
#include <Databases/IDatabase.h>
#include <Databases/DatabaseOrdinary.h>
using namespace DB;
int main(int, char **)
try
{
Poco::AutoPtr<Poco::ConsoleChannel> 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<DatabaseOrdinary>("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;
}

View File

@ -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)

View File

@ -1,228 +0,0 @@
#include <iostream>
#include <thread>
#include <atomic>
#include <Processors/IProcessor.h>
#include <Processors/ISource.h>
#include <Processors/ISink.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/ConcatProcessor.h>
#include <Processors/ForkProcessor.h>
#include <Processors/LimitTransform.h>
#include <Processors/QueueBuffer.h>
#include <Processors/printPipeline.h>
#include <Columns/ColumnsNumber.h>
#include <Common/ThreadPool.h>
#include <Common/EventCounter.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromOStream.h>
#include <Processors/Executors/PipelineExecutor.h>
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<DataTypeUInt64>(), "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<DataTypeUInt64>(), "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<DataTypeUInt64>(), "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<NumbersSource>(0, 300000);
auto header = source0->getPort().getHeader();
auto limit0 = std::make_shared<LimitTransform>(header, 10, 0);
connect(source0->getPort(), limit0->getInputPort());
auto queue = std::make_shared<QueueBuffer>(header);
connect(limit0->getOutputPort(), queue->getInputPort());
auto source1 = std::make_shared<SleepyNumbersSource>(100, 100000);
auto source2 = std::make_shared<SleepyNumbersSource>(1000, 200000);
auto source3 = std::make_shared<NumbersSource>(10, 100000);
auto limit3 = std::make_shared<LimitTransform>(header, 5, 0);
connect(source3->getPort(), limit3->getInputPort());
auto source4 = std::make_shared<NumbersSource>(10, 100000);
auto limit4 = std::make_shared<LimitTransform>(header, 5, 0);
connect(source4->getPort(), limit4->getInputPort());
auto concat = std::make_shared<ConcatProcessor>(header, 2);
connect(limit3->getOutputPort(), concat->getInputs().front());
connect(limit4->getOutputPort(), concat->getInputs().back());
auto fork = std::make_shared<ForkProcessor>(header, 2);
connect(concat->getOutputPort(), fork->getInputPort());
auto print_after_concat = std::make_shared<PrintSink>("---------- ");
connect(fork->getOutputs().back(), print_after_concat->getPort());
auto resize = std::make_shared<ResizeProcessor>(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<LimitTransform>(header, 100, 0);
connect(resize->getOutputs().front(), limit->getInputPort());
auto sink = std::make_shared<PrintSink>("");
connect(limit->getOutputPort(), sink->getPort());
WriteBufferFromOStream out(std::cout);
std::vector<ProcessorPtr> 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;
}

View File

@ -1,411 +0,0 @@
#include <iostream>
#include <thread>
#include <atomic>
#include <Processors/IProcessor.h>
#include <Processors/ISource.h>
#include <Processors/ISink.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/ConcatProcessor.h>
#include <Processors/ForkProcessor.h>
#include <Processors/LimitTransform.h>
#include <Processors/QueueBuffer.h>
#include <Processors/printPipeline.h>
#include <Columns/ColumnsNumber.h>
#include <Common/ThreadPool.h>
#include <Common/EventCounter.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromOStream.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/Transforms/AggregatingTransform.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <DataTypes/DataTypeFactory.h>
#include <Processors/Transforms/MergingAggregatedTransform.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
#include <Disks/StoragePolicy.h>
#include <Disks/DiskLocal.h>
#include <Poco/ConsoleChannel.h>
#include <Poco/AutoPtr.h>
#include <Common/CurrentThread.h>
#include <Poco/Path.h>
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<DataTypeUInt64>(), "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<bool> 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<UInt64> 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<typename TimeT = std::chrono::milliseconds>
struct Measure
{
template<typename F, typename ...Args>
static typename TimeT::rep execution(F&& func, Args&&... args)
{
auto start = std::chrono::steady_clock::now();
std::forward<decltype(func)>(func)(std::forward<Args>(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<Poco::ConsoleChannel> 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<DiskLocal>("tmp", cur_path, 0);
auto tmp_volume = std::make_shared<VolumeJBOD>("tmp", std::vector<DiskPtr>{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<NumbersSource>(0, 1, block_size, 0);
auto source2 = std::make_shared<NumbersSource>(0, 1, block_size, 0);
auto source3 = std::make_shared<NumbersSource>(0, 1, block_size, 0);
auto limit1 = std::make_shared<LimitTransform>(source1->getPort().getHeader(), num_rows, 0);
auto limit2 = std::make_shared<LimitTransform>(source2->getPort().getHeader(), num_rows, 0);
auto limit3 = std::make_shared<LimitTransform>(source3->getPort().getHeader(), num_rows, 0);
auto resize = std::make_shared<ResizeProcessor>(source1->getPort().getHeader(), 3, 1);
AggregateDescriptions aggregate_descriptions(1);
DataTypes sum_types = { std::make_shared<DataTypeUInt64>() };
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<AggregatingTransformParams>(params, /* final =*/ false);
auto merge_params = std::make_shared<AggregatingTransformParams>(params, /* final =*/ true);
auto aggregating = std::make_shared<AggregatingTransform>(source1->getPort().getHeader(), agg_params);
auto merging = std::make_shared<MergingAggregatedTransform>(aggregating->getOutputs().front().getHeader(), merge_params, 4);
auto sink = std::make_shared<CheckSink>(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<ProcessorPtr> 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<NumbersSource>(0, 1, block_size, 0);
auto source2 = std::make_shared<NumbersSource>(0, 1, block_size, 0);
auto source3 = std::make_shared<NumbersSource>(0, 1, block_size, 0);
auto limit1 = std::make_shared<LimitTransform>(source1->getPort().getHeader(), num_rows, 0);
auto limit2 = std::make_shared<LimitTransform>(source2->getPort().getHeader(), num_rows, 0);
auto limit3 = std::make_shared<LimitTransform>(source3->getPort().getHeader(), num_rows, 0);
AggregateDescriptions aggregate_descriptions(1);
DataTypes sum_types = { std::make_shared<DataTypeUInt64>() };
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<AggregatingTransformParams>(params, /* final =*/ false);
auto merge_params = std::make_shared<AggregatingTransformParams>(params, /* final =*/ true);
ManyAggregatedDataPtr data = std::make_unique<ManyAggregatedData>(3);
auto aggregating1 = std::make_shared<AggregatingTransform>(source1->getPort().getHeader(), agg_params, data, 0, 4, 4);
auto aggregating2 = std::make_shared<AggregatingTransform>(source1->getPort().getHeader(), agg_params, data, 1, 4, 4);
auto aggregating3 = std::make_shared<AggregatingTransform>(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<CheckSink>(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<ProcessorPtr> 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<String> messages;
std::vector<Int64> 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;
}

View File

@ -1,165 +0,0 @@
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Processors/IProcessor.h>
#include <Processors/ISource.h>
#include <Processors/ISink.h>
#include <Processors/ISimpleTransform.h>
#include <Processors/LimitTransform.h>
#include <Processors/printPipeline.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatSettings.h>
#include <iostream>
#include <chrono>
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<DataTypeUInt64>(), "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<DataTypeUInt64>(), "number" }}),
Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "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<DataTypeUInt64>(), "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<typename TimeT = std::chrono::milliseconds>
struct Measure
{
template<typename F, typename ...Args>
static typename TimeT::rep execution(F&& func, Args&&... args)
{
auto start = std::chrono::steady_clock::now();
std::forward<decltype(func)>(func)(std::forward<Args>(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<NumbersSource>(0, 100000);
auto transform1 = std::make_shared<SleepyTransform>(100000);
auto transform2 = std::make_shared<SleepyTransform>(100000);
auto transform3 = std::make_shared<SleepyTransform>(100000);
auto limit = std::make_shared<LimitTransform>(source->getPort().getHeader(), 20, 0);
auto sink = std::make_shared<PrintSink>("");
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<ProcessorPtr> 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;
}

View File

@ -1,285 +0,0 @@
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Processors/ISink.h>
#include <Processors/ISource.h>
#include <Processors/LimitTransform.h>
#include <Processors/printPipeline.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatSettings.h>
#include <iostream>
#include <chrono>
#include <Processors/ISimpleTransform.h>
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<DataTypeUInt64>(), "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<DataTypeUInt64>(), "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<DataTypeUInt64>(), "number" }})},
{Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "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<ExpandingProcessor>()};
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<typename TimeT = std::chrono::milliseconds>
struct Measure
{
template<typename F, typename ...Args>
static typename TimeT::rep execution(F&& func, Args&&... args)
{
auto start = std::chrono::steady_clock::now();
std::forward<decltype(func)>(func)(std::forward<Args>(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<OneNumberSource>(num);
auto expanding = std::make_shared<ExpandingProcessor>();
auto sink = std::make_shared<PrintSink>("");
connect(source->getPort(), expanding->getInputs().front());
connect(expanding->getOutputs().front(), sink->getPort());
std::vector<ProcessorPtr> 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;
}

View File

@ -1,334 +0,0 @@
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Processors/IProcessor.h>
#include <Processors/ISource.h>
#include <Processors/ISink.h>
#include <Processors/ISimpleTransform.h>
#include <Processors/LimitTransform.h>
#include <Processors/printPipeline.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatSettings.h>
#include <iostream>
#include <chrono>
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<UInt64, size_t>;
std::priority_queue<Key, std::vector<Key>, 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<size_t> positions;
std::vector<bool> 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<DataTypeUInt64>(), "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<DataTypeUInt64>(), "number" }}),
Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "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<DataTypeUInt64>(), "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<typename TimeT = std::chrono::milliseconds>
struct Measure
{
template<typename F, typename ...Args>
static typename TimeT::rep execution(F&& func, Args&&... args)
{
auto start = std::chrono::steady_clock::now();
std::forward<decltype(func)>(func)(std::forward<Args>(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<NumbersSource>(start1, 3, 100000);
auto source2 = std::make_shared<NumbersSource>(start2, 3, 100000);
auto source3 = std::make_shared<NumbersSource>(start3, 3, 100000);
auto transform1 = std::make_shared<SleepyTransform>(100000);
auto transform2 = std::make_shared<SleepyTransform>(100000);
auto transform3 = std::make_shared<SleepyTransform>(100000);
auto limit1 = std::make_shared<LimitTransform>(source1->getPort().getHeader(), 20, 0);
auto limit2 = std::make_shared<LimitTransform>(source2->getPort().getHeader(), 20, 0);
auto limit3 = std::make_shared<LimitTransform>(source3->getPort().getHeader(), 20, 0);
auto merge = std::make_shared<MergingSortedProcessor>(source1->getPort().getHeader(), 3);
auto limit_fin = std::make_shared<LimitTransform>(source1->getPort().getHeader(), 54, 0);
auto sink = std::make_shared<PrintSink>("");
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<ProcessorPtr> 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;
}

View File

@ -1,250 +0,0 @@
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Disks/StoragePolicy.h>
#include <Disks/DiskLocal.h>
#include <Processors/IProcessor.h>
#include <Processors/ISource.h>
#include <Processors/ISink.h>
#include <Processors/ISimpleTransform.h>
#include <Processors/LimitTransform.h>
#include <Processors/printPipeline.h>
#include <Processors/Transforms/MergeSortingTransform.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatSettings.h>
#include <iostream>
#include <chrono>
#include <Poco/ConsoleChannel.h>
#include <Poco/AutoPtr.h>
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<DataTypeUInt64>(), "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<DataTypeUInt64>(), "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<typename TimeT = std::chrono::milliseconds>
struct Measure
{
template<typename F, typename ...Args>
static typename TimeT::rep execution(F&& func, Args&&... args)
{
auto start = std::chrono::steady_clock::now();
std::forward<decltype(func)>(func)(std::forward<Args>(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<Poco::ConsoleChannel> channel = new Poco::ConsoleChannel(std::cerr);
Poco::Logger::root().setChannel(channel);
Poco::Logger::root().setLevel("trace");
auto disk = std::make_shared<DiskLocal>("tmp", ".", 0);
auto tmp_volume = std::make_shared<VolumeJBOD>("tmp", std::vector<DiskPtr>{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<NumbersSource>(blocks_count, source_block_size, 100);
SortDescription description = {{0, 1, 1}};
auto transform = std::make_shared<MergeSortingTransform>(
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<CheckSortedSink>();
connect(source->getPort(), transform->getInputs().front());
connect(transform->getOutputs().front(), sink->getPort());
std::vector<ProcessorPtr> processors = {source, transform, sink};
PipelineExecutor executor(processors);
executor.execute(num_threads);
WriteBufferFromOStream out(std::cout);
printPipeline(executor.getProcessors(), out);
};
std::map<std::string, Int64> 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;
}

View File

@ -1,207 +0,0 @@
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Processors/IProcessor.h>
#include <Processors/ISource.h>
#include <Processors/ISink.h>
#include <Processors/ISimpleTransform.h>
#include <Processors/LimitTransform.h>
#include <Processors/printPipeline.h>
#include <Processors/Merges/MergingSortedTransform.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatSettings.h>
#include <iostream>
#include <chrono>
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<DataTypeUInt64>(), "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<DataTypeUInt64>(), "number" }}),
Block({ColumnWithTypeAndName{ ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "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<DataTypeUInt64>(), "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<typename TimeT = std::chrono::milliseconds>
struct Measure
{
template<typename F, typename ...Args>
static typename TimeT::rep execution(F&& func, Args&&... args)
{
auto start = std::chrono::steady_clock::now();
std::forward<decltype(func)>(func)(std::forward<Args>(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<NumbersSource>(start1, 3, 2, 100000);
auto source2 = std::make_shared<NumbersSource>(start2, 3, 2, 100000);
auto source3 = std::make_shared<NumbersSource>(start3, 3, 2, 100000);
auto transform1 = std::make_shared<SleepyTransform>(100000);
auto transform2 = std::make_shared<SleepyTransform>(100000);
auto transform3 = std::make_shared<SleepyTransform>(100000);
auto limit1 = std::make_shared<LimitTransform>(source1->getPort().getHeader(), 20, 0);
auto limit2 = std::make_shared<LimitTransform>(source2->getPort().getHeader(), 20, 0);
auto limit3 = std::make_shared<LimitTransform>(source3->getPort().getHeader(), 20, 0);
SortDescription description = {{0, 1, 1}};
auto merge = std::make_shared<MergingSortedTransform>(source1->getPort().getHeader(), 3, description, 2);
auto limit_fin = std::make_shared<LimitTransform>(source1->getPort().getHeader(), 54, 0);
auto sink = std::make_shared<PrintSink>("");
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<ProcessorPtr> 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;
}

View File

@ -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> & 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);

View File

@ -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> & 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> context;
Names column_names;
UInt64 max_block_size;

View File

@ -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> & 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)

View File

@ -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> & context_);
Block getHeader() const override;
@ -22,7 +22,7 @@ public:
private:
StorageKafka & storage;
Context context;
const std::shared_ptr<Context> context;
ProducerBufferPtr buffer;
BlockOutputStreamPtr child;
};

View File

@ -1,7 +1,7 @@
#pragma once
#include <Core/SettingsCollection.h>
#include <Core/Settings.h>
namespace DB
{
@ -15,18 +15,34 @@ struct KafkaSettings : public SettingsCollection<KafkaSettings>
{
#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)

View File

@ -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<KafkaSettings> 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<Context>(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>(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<SourceFromInputStream>(std::make_shared<KafkaBlockInputStream>(*this, context, column_names, 1)));
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::make_shared<KafkaBlockInputStream>(*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>(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<KafkaBlockOutputStream>(*this, context);
return std::make_shared<KafkaBlockOutputStream>(*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<cppkafka::Consumer>(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<ReadBufferFromKafkaConsumer>(consumer, log, batch_size, poll_timeout, intermediate_commit, stream_cancelled, getTopics());
return std::make_shared<ReadBufferFromKafkaConsumer>(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<ASTInsertQuery>();
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<KafkaSettings>();
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<ASTLiteral &>().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<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
{
brokers = safeGet<String>(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<ASTLiteral &>().value.safeGet<String>();
}
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<ASTLiteral &>().value.safeGet<String>();
}
// 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<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
{
format = safeGet<String>(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<ASTLiteral>();
String arg;
if (ast && ast->value.getType() == Field::Types::String)
{
arg = safeGet<String>(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<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
{
schema = safeGet<String>(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<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64)
{
num_consumers = safeGet<UInt64>(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<size_t>(kafka_settings.kafka_max_block_size);
if (args_count >= 8)
{
const auto * ast = engine_args[7]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64)
{
max_block_size = static_cast<size_t>(safeGet<UInt64>(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<size_t>(kafka_settings.kafka_skip_broken_messages);
if (args_count >= 9)
{
const auto * ast = engine_args[8]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64)
{
skip_broken = static_cast<size_t>(safeGet<UInt64>(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<bool>(kafka_settings.kafka_commit_every_batch);
if (args_count >= 10)
{
const auto * ast = engine_args[9]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64)
{
intermediate_commit = static_cast<bool>(safeGet<UInt64>(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, });

View File

@ -3,6 +3,7 @@
#include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h>
#include <Storages/Kafka/Buffer_fwd.h>
#include <Storages/Kafka/KafkaSettings.h>
#include <Interpreters/Context.h>
#include <Poco/Semaphore.h>
@ -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<KafkaSettings> kafka_settings_);
private:
// Configuration and state
Context global_context;
Context kafka_context;
Names topics;
Context & global_context;
std::shared_ptr<Context> kafka_context;
std::unique_ptr<KafkaSettings> 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<ConsumerBufferPtr> buffers; /// available buffers for Kafka consumers
size_t skip_broken;
bool intermediate_commit;
std::mutex mutex;
// Stream thread
BackgroundSchedulePool::TaskHolder task;
std::atomic<bool> 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);
};

View File

@ -1,5 +1,4 @@
#include <Compression/CompressedReadBuffer.h>
#include <DataStreams/ExpressionBlockInputStream.h>
#include <DataStreams/copyData.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDate.h>

View File

@ -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 <macros> 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;

View File

@ -15,7 +15,6 @@
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeString.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/UnionBlockInputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/narrowBlockInputStreams.h>
@ -28,6 +27,7 @@
#include <Processors/Sources/SourceWithProgress.h>
#include <Processors/Pipe.h>
namespace DB
{
namespace ErrorCodes

View File

@ -35,6 +35,8 @@
#include <Processors/Sources/SourceWithProgress.h>
#include <Processors/Pipe.h>
#include <sstream>
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())

View File

@ -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)

View File

@ -1,113 +0,0 @@
#include <iostream>
#include <IO/WriteBufferFromOStream.h>
#include <Storages/StorageLog.h>
#include <Formats/FormatFactory.h>
#include <DataStreams/LimitBlockInputStream.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/copyData.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <Interpreters/Context.h>
#include <Common/typeid_cast.h>
#include <Disks/DiskLocal.h>
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
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<DataTypeUInt64>());
names_and_types.emplace_back("b", std::make_shared<DataTypeUInt8>());
SharedContextHolder shared_context = Context::createShared();
auto context = Context::createGlobal(shared_context.get());
context.makeGlobalContext();
context.setPath("./");
DiskPtr disk = std::make_unique<DiskLocal>("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<ColumnUInt64 &>(*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<ColumnUInt8 &>(*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<TreeExecutorBlockInputStream>(std::move(table->read(column_names, {}, context, stage, 8192, 1)[0]));
Block sample;
{
ColumnWithTypeAndName col;
col.type = std::make_shared<DataTypeUInt64>();
sample.insert(std::move(col));
}
{
ColumnWithTypeAndName col;
col.type = std::make_shared<DataTypeUInt8>();
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;
}

View File

@ -1,47 +0,0 @@
#include <iostream>
#include <IO/WriteBufferFromOStream.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <DataStreams/LimitBlockInputStream.h>
#include <Formats/FormatFactory.h>
#include <DataStreams/copyData.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
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<DataTypeUInt64>();
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<TreeExecutorBlockInputStream>(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;
}

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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)

View File

@ -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):

View File

@ -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\')

View File

@ -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;

View File

@ -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\\\\]\')

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -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