This commit is contained in:
Roman Peshkurov 2015-06-05 19:26:55 +03:00
commit a5537c31a7
108 changed files with 5043 additions and 1336 deletions

View File

@ -12,8 +12,6 @@
#include <DB/Core/Protocol.h>
#include <DB/Core/QueryProcessingStage.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/DataStreams/IBlockInputStream.h>
#include <DB/DataStreams/IBlockOutputStream.h>
#include <DB/DataStreams/BlockStreamProfileInfo.h>
@ -50,7 +48,6 @@ class Connection : private boost::noncopyable
public:
Connection(const String & host_, UInt16 port_, const String & default_database_,
const String & user_, const String & password_,
const DataTypeFactory & data_type_factory_,
const String & client_name_ = "client",
Protocol::Compression::Enum compression_ = Protocol::Compression::Enable,
Poco::Timespan connect_timeout_ = Poco::Timespan(DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, 0),
@ -59,17 +56,47 @@ public:
Poco::Timespan ping_timeout_ = Poco::Timespan(DBMS_DEFAULT_PING_TIMEOUT_SEC, 0))
:
host(host_), port(port_), default_database(default_database_),
user(user_), password(password_),
user(user_), password(password_), resolved_address(host, port),
client_name(client_name_),
compression(compression_), data_type_factory(data_type_factory_),
compression(compression_),
connect_timeout(connect_timeout_), receive_timeout(receive_timeout_), send_timeout(send_timeout_),
ping_timeout(ping_timeout_),
log_wrapper(host, port)
log_wrapper(*this)
{
/// Соединеняемся не сразу, а при первой необходимости.
if (user.empty())
user = "default";
setDescription();
}
Connection(const String & host_, UInt16 port_, const Poco::Net::SocketAddress & resolved_address_,
const String & default_database_,
const String & user_, const String & password_,
const String & client_name_ = "client",
Protocol::Compression::Enum compression_ = Protocol::Compression::Enable,
Poco::Timespan connect_timeout_ = Poco::Timespan(DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, 0),
Poco::Timespan receive_timeout_ = Poco::Timespan(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, 0),
Poco::Timespan send_timeout_ = Poco::Timespan(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0),
Poco::Timespan ping_timeout_ = Poco::Timespan(DBMS_DEFAULT_PING_TIMEOUT_SEC, 0))
:
host(host_), port(port_),
default_database(default_database_),
user(user_), password(password_),
resolved_address(resolved_address_),
client_name(client_name_),
compression(compression_),
connect_timeout(connect_timeout_), receive_timeout(receive_timeout_), send_timeout(send_timeout_),
ping_timeout(ping_timeout_),
log_wrapper(*this)
{
/// Соединеняемся не сразу, а при первой необходимости.
if (user.empty())
user = "default";
setDescription();
}
virtual ~Connection() {};
@ -99,8 +126,21 @@ public:
void getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & revision);
/// Адрес сервера - для сообщений в логе и в эксепшенах.
String getServerAddress() const;
/// Для сообщений в логе и в эксепшенах.
const String & getDescription() const
{
return description;
}
const String & getHost() const
{
return host;
}
UInt16 getPort() const
{
return port;
}
/// Если последний флаг true, то затем необходимо вызвать sendExternalTablesData
void sendQuery(const String & query, const String & query_id_ = "", UInt64 stage = QueryProcessingStage::Complete,
@ -134,16 +174,6 @@ public:
*/
void disconnect();
const std::string & getHost() const
{
return host;
}
UInt16 getPort() const
{
return port;
}
size_t outBytesCount() const { return !out.isNull() ? out->count() : 0; }
size_t inBytesCount() const { return !in.isNull() ? in->count() : 0; }
@ -154,6 +184,15 @@ private:
String user;
String password;
/** Адрес может быть заранее отрезолвен и передан в конструктор. Тогда поля host и port имеют смысл только для логгирования.
* Иначе адрес резолвится в конструкторе. То есть, DNS балансировка не поддерживается.
*/
Poco::Net::SocketAddress resolved_address;
/// Для сообщений в логе и в эксепшенах.
String description;
void setDescription();
String client_name;
bool connected = false;
@ -169,8 +208,8 @@ private:
String query_id;
UInt64 compression; /// Сжимать ли данные при взаимодействии с сервером.
const DataTypeFactory & data_type_factory;
/// каким алгоритмом сжимать данные при INSERT и данные внешних таблиц
CompressionMethod network_compression_method = CompressionMethod::LZ4;
/** Если не nullptr, то используется, чтобы ограничить сетевой трафик.
* Учитывается только трафик при передаче блоков. Другие пакеты не учитываются.
@ -194,22 +233,22 @@ private:
class LoggerWrapper
{
public:
LoggerWrapper(std::string & host_, size_t port_) : log(nullptr), host(host_), port(port_)
LoggerWrapper(Connection & parent_)
: log(nullptr), parent(parent_)
{
}
Logger * get()
{
if (!log)
log = &Logger::get("Connection (" + Poco::Net::SocketAddress(host, port).toString() + ")");
log = &Logger::get("Connection (" + parent.getDescription() + ")");
return log;
}
private:
std::atomic<Logger *> log;
std::string host;
size_t port;
Connection & parent;
};
LoggerWrapper log_wrapper;

View File

@ -54,18 +54,35 @@ public:
typedef PoolBase<Connection> Base;
ConnectionPool(unsigned max_connections_,
const String & host_, UInt16 port_, const String & default_database_,
const String & host_, UInt16 port_,
const String & default_database_,
const String & user_, const String & password_,
const DataTypeFactory & data_type_factory_,
const String & client_name_ = "client",
Protocol::Compression::Enum compression_ = Protocol::Compression::Enable,
Poco::Timespan connect_timeout_ = Poco::Timespan(DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, 0),
Poco::Timespan receive_timeout_ = Poco::Timespan(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, 0),
Poco::Timespan send_timeout_ = Poco::Timespan(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0))
: Base(max_connections_, &Logger::get("ConnectionPool (" + Poco::Net::SocketAddress(host_, port_).toString() + ")")),
: Base(max_connections_, &Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
host(host_), port(port_), default_database(default_database_),
user(user_), password(password_),
client_name(client_name_), compression(compression_), data_type_factory(data_type_factory_),
user(user_), password(password_), resolved_address(host_, port_),
client_name(client_name_), compression(compression_),
connect_timeout(connect_timeout_), receive_timeout(receive_timeout_), send_timeout(send_timeout_)
{
}
ConnectionPool(unsigned max_connections_,
const String & host_, UInt16 port_, const Poco::Net::SocketAddress & resolved_address_,
const String & default_database_,
const String & user_, const String & password_,
const String & client_name_ = "client",
Protocol::Compression::Enum compression_ = Protocol::Compression::Enable,
Poco::Timespan connect_timeout_ = Poco::Timespan(DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, 0),
Poco::Timespan receive_timeout_ = Poco::Timespan(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, 0),
Poco::Timespan send_timeout_ = Poco::Timespan(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0))
: Base(max_connections_, &Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
host(host_), port(port_), default_database(default_database_),
user(user_), password(password_), resolved_address(resolved_address_),
client_name(client_name_), compression(compression_),
connect_timeout(connect_timeout_), receive_timeout(receive_timeout_), send_timeout(send_timeout_)
{
}
@ -90,8 +107,9 @@ protected:
ConnectionPtr allocObject() override
{
return new Connection(
host, port, default_database, user, password,
data_type_factory, client_name, compression,
host, port, resolved_address,
default_database, user, password,
client_name, compression,
connect_timeout, receive_timeout, send_timeout);
}
@ -102,11 +120,14 @@ private:
String user;
String password;
/** Адрес может быть заранее отрезолвен и передан в конструктор. Тогда поля host и port имеют смысл только для логгирования.
* Иначе адрес резолвится в конструкторе. То есть, DNS балансировка не поддерживается.
*/
Poco::Net::SocketAddress resolved_address;
String client_name;
Protocol::Compression::Enum compression; /// Сжимать ли данные при взаимодействии с сервером.
const DataTypeFactory & data_type_factory;
Poco::Timespan connect_timeout;
Poco::Timespan receive_timeout;
Poco::Timespan send_timeout;

View File

@ -5,6 +5,7 @@
#include <statdaemons/PoolWithFailoverBase.h>
#include <DB/Common/getFQDNOrHostName.h>
#include <DB/Client/ConnectionPool.h>
@ -33,7 +34,7 @@ public:
: Base(nested_pools_, max_tries_, decrease_error_period_,
&Logger::get("ConnectionPoolWithFailover")), default_load_balancing(load_balancing)
{
std::string local_hostname = Poco::Net::DNS::hostName();
const std::string & local_hostname = getFQDNOrHostName();
hostname_differences.resize(nested_pools.size());
for (size_t i = 0; i < nested_pools.size(); ++i)

View File

@ -4,6 +4,7 @@
#include <boost/algorithm/string.hpp>
#include <DB/DataStreams/AsynchronousBlockInputStream.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/Interpreters/Context.h>
#include <DB/IO/copyData.h>
#include <DB/IO/ReadBufferFromIStream.h>
@ -42,11 +43,13 @@ public:
/// Инициализировать sample_block по структуре таблицы сохраненной в structure
virtual void initSampleBlock(const Context & context)
{
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
for (size_t i = 0; i < structure.size(); ++i)
{
ColumnWithNameAndType column;
column.name = structure[i].first;
column.type = context.getDataTypeFactory().get(structure[i].second);
column.type = data_type_factory.get(structure[i].second);
column.column = column.type->createColumn();
sample_block.insert(column);
}
@ -58,7 +61,7 @@ public:
initReadBuffer();
initSampleBlock(context);
ExternalTableData res = std::make_pair(new AsynchronousBlockInputStream(context.getFormatFactory().getInput(
format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE, context.getDataTypeFactory())), name);
format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE)), name);
return res;
}

View File

@ -192,6 +192,8 @@ public:
return *this;
}
T * data() { return t_start(); }
const T * data() const { return t_start(); }
size_t size() const { return t_end() - t_start(); }
bool empty() const { return t_end() == t_start(); }

View File

@ -0,0 +1,8 @@
#pragma once
#include <string>
/** Получить FQDN для локального сервера путём DNS-резолвинга hostname - аналогично вызову утилиты hostname с флагом -f.
* Если не получилось отрезолвить, то вернуть hostname - аналогично вызову утилиты hostname без флагов или uname -n.
*/
const std::string & getFQDNOrHostName();

View File

@ -7,6 +7,7 @@
#define DBMS_DEFAULT_HOST "localhost"
#define DBMS_DEFAULT_PORT 9000
#define DBMS_DEFAULT_PORT_STR "9000"
#define DBMS_DEFAULT_HTTP_PORT 8123
#define DBMS_DEFAULT_CONNECT_TIMEOUT_SEC 10
#define DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS 50
#define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300

View File

@ -285,6 +285,7 @@ namespace ErrorCodes
LEADERSHIP_LOST = 278,
ALL_CONNECTION_TRIES_FAILED = 279,
KEEPER_EXCEPTION = 999,
POCO_EXCEPTION = 1000,
STD_EXCEPTION = 1001,
UNKNOWN_EXCEPTION = 1002,

View File

@ -9,7 +9,6 @@
#include <sparsehash/dense_hash_map>
#include <DB/DataTypes/IDataType.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/IO/ReadBufferFromString.h>
#include "Names.h"
@ -45,11 +44,11 @@ class NamesAndTypesList : public std::list<NameAndTypePair>
public:
using std::list<NameAndTypePair>::list;
void readText(ReadBuffer & buf, const DataTypeFactory & data_type_factory);
void readText(ReadBuffer & buf);
void writeText(WriteBuffer & buf) const;
String toString() const;
static NamesAndTypesList parse(const String & s, const DataTypeFactory & data_type_factory);
static NamesAndTypesList parse(const String & s);
/// Все элементы rhs должны быть различны.
bool isSubsetOf(const NamesAndTypesList & rhs) const;

View File

@ -1,7 +1,5 @@
#pragma once
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/DataStreams/IBlockInputStream.h>
#include <DB/DataStreams/IBlockOutputStream.h>
@ -16,8 +14,8 @@ class FormatFactory
{
public:
BlockInputStreamPtr getInput(const String & name, ReadBuffer & buf,
Block & sample, size_t max_block_size, const DataTypeFactory & data_type_factory) const;
Block & sample, size_t max_block_size) const;
BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf,
Block & sample) const;
};

View File

@ -3,7 +3,6 @@
#include <DB/Core/Progress.h>
#include <DB/Interpreters/Limits.h>
#include <DB/Interpreters/Quota.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/DataStreams/BlockStreamProfileInfo.h>
@ -14,6 +13,8 @@
namespace DB
{
class QuotaForIntervals;
/** Смотрит за тем, как работает источник блоков.
* Позволяет получить информацию для профайлинга:

View File

@ -66,9 +66,9 @@ public:
/// limit - если не 0, то можно выдать только первые limit строк в сортированном порядке.
MergeSortingBlockInputStream(BlockInputStreamPtr input_, SortDescription & description_,
size_t max_merged_block_size_, size_t limit_,
size_t max_bytes_before_external_sort_, const std::string & tmp_path_, const DataTypeFactory & data_type_factory_)
size_t max_bytes_before_external_sort_, const std::string & tmp_path_)
: description(description_), max_merged_block_size(max_merged_block_size_), limit(limit_),
max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_path(tmp_path_), data_type_factory(data_type_factory_)
max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_path(tmp_path_)
{
children.push_back(input_);
}
@ -97,7 +97,6 @@ private:
size_t max_bytes_before_external_sort;
const std::string tmp_path;
const DataTypeFactory & data_type_factory;
Logger * log = &Logger::get("MergeSortingBlockInputStream");
@ -115,8 +114,8 @@ private:
CompressedReadBuffer compressed_in;
BlockInputStreamPtr block_in;
TemporaryFileStream(const std::string & path, const DataTypeFactory & data_type_factory)
: file_in(path), compressed_in(file_in), block_in(new NativeBlockInputStream(compressed_in, data_type_factory)) {}
TemporaryFileStream(const std::string & path)
: file_in(path), compressed_in(file_in), block_in(new NativeBlockInputStream(compressed_in)) {}
};
std::vector<std::unique_ptr<TemporaryFileStream>> temporary_inputs;

View File

@ -1,6 +1,5 @@
#pragma once
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
@ -16,8 +15,8 @@ public:
/** В случае указания ненулевой server_revision, может ожидаться и считываться дополнительная информация о блоке,
* в зависимости от поддерживаемой для указанной ревизии.
*/
NativeBlockInputStream(ReadBuffer & istr_, const DataTypeFactory & data_type_factory_, UInt64 server_revision_ = 0)
: istr(istr_), data_type_factory(data_type_factory_), server_revision(server_revision_) {}
NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_ = 0)
: istr(istr_), server_revision(server_revision_) {}
String getName() const override { return "NativeBlockInputStream"; }
@ -35,7 +34,6 @@ protected:
private:
ReadBuffer & istr;
const DataTypeFactory & data_type_factory;
UInt64 server_revision;
};

View File

@ -128,8 +128,10 @@ private:
* все элементы - нулевые.
*/
template<class TSortCursor>
void mergeMaps(Row & row, TSortCursor & cursor)
bool mergeMaps(Row & row, TSortCursor & cursor)
{
auto non_empty_map_present = false;
/// merge nested maps
for (const auto & map : maps_to_sum)
{
@ -235,11 +237,26 @@ private:
else
break;
/// discard last row if necessary
if (discard_prev)
key_array_result.pop_back();
/// store results into accumulator-row
key_array_lhs = std::move(key_array_result);
for (const auto val_col_index : ext::range(0, val_count))
{
/// discard last row if necessary
if (discard_prev)
val_arrays_result[val_col_index].pop_back();
row[map.val_col_nums[val_col_index]].get<Array>() = std::move(val_arrays_result[val_col_index]);
}
if (!key_array_lhs.empty())
non_empty_map_present = true;
}
return non_empty_map_present;
}
/** Прибавить строчку под курсором к row.
@ -248,9 +265,7 @@ private:
template<class TSortCursor>
bool addRow(Row & row, TSortCursor & cursor)
{
mergeMaps(row, cursor);
bool res = false; /// Есть ли хотя бы одно ненулевое число.
bool res = mergeMaps(row, cursor); /// Есть ли хотя бы одно ненулевое число или непустой массив
for (size_t i = 0, size = column_numbers_to_sum.size(); i < size; ++i)
{

View File

@ -20,32 +20,37 @@ const auto max_connections = 16;
class ClickHouseDictionarySource final : public IDictionarySource
{
public:
ClickHouseDictionarySource(const Poco::Util::AbstractConfiguration & config,
ClickHouseDictionarySource(const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block, Context & context)
: host{config.getString(config_prefix + ".host")},
: dict_struct{dict_struct},
host{config.getString(config_prefix + ".host")},
port(config.getInt(config_prefix + ".port")),
user{config.getString(config_prefix + ".user", "")},
password{config.getString(config_prefix + ".password", "")},
db{config.getString(config_prefix + ".db", "")},
table{config.getString(config_prefix + ".table")},
where{config.getString(config_prefix + ".where", "")},
sample_block{sample_block}, context(context),
is_local{isLocalAddress({ host, port })},
pool{is_local ? nullptr : std::make_unique<ConnectionPool>(
max_connections, host, port, db, user, password, context.getDataTypeFactory(),
max_connections, host, port, db, user, password,
"ClickHouseDictionarySource")
},
load_all_query{composeLoadAllQuery(sample_block, db, table)}
load_all_query{composeLoadAllQuery()}
{}
/// copy-constructor is provided in order to support cloneability
ClickHouseDictionarySource(const ClickHouseDictionarySource & other)
: host{other.host}, port{other.port}, user{other.user}, password{other.password},
: dict_struct{other.dict_struct},
host{other.host}, port{other.port}, user{other.user}, password{other.password},
db{other.db}, table{other.table},
where{other.where},
sample_block{other.sample_block}, context(other.context),
is_local{other.is_local},
pool{is_local ? nullptr : std::make_unique<ConnectionPool>(
max_connections, host, port, db, user, password, context.getDataTypeFactory(),
max_connections, host, port, db, user, password,
"ClickHouseDictionarySource")},
load_all_query{other.load_all_query}
{}
@ -60,7 +65,7 @@ public:
return new RemoteBlockInputStream{pool.get(), load_all_query, nullptr};
}
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> ids) override
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
{
const auto query = composeLoadIdsQuery(ids);
@ -74,10 +79,13 @@ public:
DictionarySourcePtr clone() const override { return std::make_unique<ClickHouseDictionarySource>(*this); }
std::string toString() const override { return "ClickHouse: " + db + '.' + table; }
std::string toString() const override
{
return "ClickHouse: " + db + '.' + table + (where.empty() ? "" : ", where: " + where);
}
private:
static std::string composeLoadAllQuery(const Block & block, const std::string & db, const std::string & table)
std::string composeLoadAllQuery() const
{
std::string query;
@ -85,14 +93,19 @@ private:
WriteBufferFromString out{query};
writeString("SELECT ", out);
auto first = true;
for (const auto idx : ext::range(0, block.columns()))
{
if (!first)
writeString(", ", out);
writeProbablyBackQuotedString(dict_struct.id_name, out);
writeString(block.getByPosition(idx).name, out);
first = false;
for (const auto & attr : dict_struct.attributes)
{
writeString(", ", out);
if (!attr.expression.empty())
{
writeString(attr.expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(attr.name, out);
}
writeString(" FROM ", out);
@ -102,6 +115,13 @@ private:
writeChar('.', out);
}
writeProbablyBackQuotedString(table, out);
if (!where.empty())
{
writeString(" WHERE ", out);
writeString(where, out);
}
writeChar(';', out);
}
@ -116,17 +136,21 @@ private:
WriteBufferFromString out{query};
writeString("SELECT ", out);
auto first = true;
for (const auto idx : ext::range(0, sample_block.columns()))
{
if (!first)
writeString(", ", out);
writeProbablyBackQuotedString(dict_struct.id_name, out);
writeString(sample_block.getByPosition(idx).name, out);
first = false;
for (const auto & attr : dict_struct.attributes)
{
writeString(", ", out);
if (!attr.expression.empty())
{
writeString(attr.expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(attr.name, out);
}
const auto & id_column_name = sample_block.getByPosition(0).name;
writeString(" FROM ", out);
if (!db.empty())
{
@ -134,11 +158,19 @@ private:
writeChar('.', out);
}
writeProbablyBackQuotedString(table, out);
writeString(" WHERE ", out);
writeProbablyBackQuotedString(id_column_name, out);
if (!where.empty())
{
writeString(where, out);
writeString(" AND ", out);
}
writeProbablyBackQuotedString(dict_struct.id_name, out);
writeString(" IN (", out);
first = true;
auto first = true;
for (const auto id : ids)
{
if (!first)
@ -154,12 +186,14 @@ private:
return query;
}
const DictionaryStructure dict_struct;
const std::string host;
const UInt16 port;
const std::string user;
const std::string password;
const std::string db;
const std::string table;
const std::string where;
Block sample_block;
Context & context;
const bool is_local;

View File

@ -64,11 +64,11 @@ public:
}
else if ("mysql" == source_type)
{
return std::make_unique<MySQLDictionarySource>(config, config_prefix + ".mysql", sample_block);
return std::make_unique<MySQLDictionarySource>(dict_struct, config, config_prefix + ".mysql", sample_block);
}
else if ("clickhouse" == source_type)
{
return std::make_unique<ClickHouseDictionarySource>(config, config_prefix + ".clickhouse",
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse",
sample_block, context);
}

View File

@ -78,7 +78,7 @@ inline std::string toString(const AttributeUnderlyingType type)
}
/// Min and max lifetimes for a dictionary or it's entry
struct DictionaryLifetime
struct DictionaryLifetime final
{
std::uint64_t min_sec;
std::uint64_t max_sec;
@ -101,18 +101,19 @@ struct DictionaryLifetime
* - hierarchical, whether this attribute defines a hierarchy;
* - injective, whether the mapping to parent is injective (can be used for optimization of GROUP BY?)
*/
struct DictionaryAttribute
struct DictionaryAttribute final
{
std::string name;
AttributeUnderlyingType underlying_type;
DataTypePtr type;
Field null_value;
bool hierarchical;
bool injective;
const std::string name;
const AttributeUnderlyingType underlying_type;
const DataTypePtr type;
const std::string expression;
const Field null_value;
const bool hierarchical;
const bool injective;
};
/// Name of identifier plus list of attributes
struct DictionaryStructure
struct DictionaryStructure final
{
std::string id_name;
std::vector<DictionaryAttribute> attributes;
@ -142,6 +143,8 @@ struct DictionaryStructure
const auto type = DataTypeFactory::instance().get(type_string);
const auto underlying_type = getAttributeUnderlyingType(type_string);
const auto expression = config.getString(prefix + "expression", "");
const auto null_value_string = config.getString(prefix + "null_value");
Field null_value;
try
@ -174,7 +177,7 @@ struct DictionaryStructure
has_hierarchy = has_hierarchy || hierarchical;
attributes.emplace_back(DictionaryAttribute{
name, underlying_type, type, null_value, hierarchical, injective
name, underlying_type, type, expression, null_value, hierarchical, injective
});
}

View File

@ -34,13 +34,13 @@ public:
{
auto in_ptr = std::make_unique<ReadBufferFromFile>(filename);
auto stream = context.getFormatFactory().getInput(
format, *in_ptr, sample_block, max_block_size, context.getDataTypeFactory());
format, *in_ptr, sample_block, max_block_size);
last_modification = getLastModification();
return new OwningBufferBlockInputStream{stream, std::move(in_ptr)};
}
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> ids) override
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
{
throw Exception{
"Method unsupported",

View File

@ -25,7 +25,7 @@ public:
virtual bool supportsSelectiveLoad() const = 0;
/// returns an input stream with the data for a collection of identifiers
virtual BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> ids) = 0;
virtual BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) = 0;
/// indicates whether the source has been modified since last load* operation
virtual bool isModified() const = 0;

View File

@ -16,20 +16,25 @@ class MySQLDictionarySource final : public IDictionarySource
static const auto max_block_size = 8192;
public:
MySQLDictionarySource(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
MySQLDictionarySource(const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
Block & sample_block)
: db{config.getString(config_prefix + ".db", "")},
: dict_struct{dict_struct},
db{config.getString(config_prefix + ".db", "")},
table{config.getString(config_prefix + ".table")},
where{config.getString(config_prefix + ".where", "")},
sample_block{sample_block},
pool{config, config_prefix},
load_all_query{composeLoadAllQuery(sample_block, db, table)},
load_all_query{composeLoadAllQuery()},
last_modification{getLastModification()}
{}
/// copy-constructor is provided in order to support cloneability
MySQLDictionarySource(const MySQLDictionarySource & other)
: db{other.db},
: dict_struct{other.dict_struct},
db{other.db},
table{other.table},
where{other.where},
sample_block{other.sample_block},
pool{other.pool},
load_all_query{other.load_all_query}, last_modification{other.last_modification}
@ -41,7 +46,7 @@ public:
return new MySQLBlockInputStream{pool.Get(), load_all_query, sample_block, max_block_size};
}
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> ids) override
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
{
last_modification = getLastModification();
const auto query = composeLoadIdsQuery(ids);
@ -54,7 +59,10 @@ public:
DictionarySourcePtr clone() const override { return std::make_unique<MySQLDictionarySource>(*this); }
std::string toString() const override { return "MySQL: " + db + '.' + table; }
std::string toString() const override
{
return "MySQL: " + db + '.' + table + (where.empty() ? "" : ", where: " + where);
}
private:
mysqlxx::DateTime getLastModification() const
@ -88,7 +96,7 @@ private:
return update_time;
}
static std::string composeLoadAllQuery(const Block & block, const std::string & db, const std::string & table)
std::string composeLoadAllQuery() const
{
std::string query;
@ -96,14 +104,19 @@ private:
WriteBufferFromString out{query};
writeString("SELECT ", out);
auto first = true;
for (const auto idx : ext::range(0, block.columns()))
{
if (!first)
writeString(", ", out);
writeProbablyBackQuotedString(dict_struct.id_name, out);
writeString(block.getByPosition(idx).name, out);
first = false;
for (const auto & attr : dict_struct.attributes)
{
writeString(", ", out);
if (!attr.expression.empty())
{
writeString(attr.expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(attr.name, out);
}
writeString(" FROM ", out);
@ -113,13 +126,20 @@ private:
writeChar('.', out);
}
writeProbablyBackQuotedString(table, out);
if (!where.empty())
{
writeString(" WHERE ", out);
writeString(where, out);
}
writeChar(';', out);
}
return query;
}
std::string composeLoadIdsQuery(const std::vector<std::uint64_t> ids)
std::string composeLoadIdsQuery(const std::vector<std::uint64_t> & ids)
{
std::string query;
@ -127,17 +147,21 @@ private:
WriteBufferFromString out{query};
writeString("SELECT ", out);
auto first = true;
for (const auto idx : ext::range(0, sample_block.columns()))
{
if (!first)
writeString(", ", out);
writeProbablyBackQuotedString(dict_struct.id_name, out);
writeString(sample_block.getByPosition(idx).name, out);
first = false;
for (const auto & attr : dict_struct.attributes)
{
writeString(", ", out);
if (!attr.expression.empty())
{
writeString(attr.expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(attr.name, out);
}
const auto & id_column_name = sample_block.getByPosition(0).name;
writeString(" FROM ", out);
if (!db.empty())
{
@ -145,11 +169,19 @@ private:
writeChar('.', out);
}
writeProbablyBackQuotedString(table, out);
writeString(" WHERE ", out);
writeProbablyBackQuotedString(id_column_name, out);
if (!where.empty())
{
writeString(where, out);
writeString(" AND ", out);
}
writeProbablyBackQuotedString(dict_struct.id_name, out);
writeString(" IN (", out);
first = true;
auto first = true;
for (const auto id : ids)
{
if (!first)
@ -165,8 +197,10 @@ private:
return query;
}
const DictionaryStructure dict_struct;
const std::string db;
const std::string table;
const std::string where;
Block sample_block;
mutable mysqlxx::PoolWithFailover pool;
const std::string load_all_query;

View File

@ -17,8 +17,8 @@ namespace DB
/** Функции работы с датой и временем.
*
* toYear, toMonth, toDayOfMonth, toDayOfWeek, toHour, toMinute, toSecond,
* toMonday, toStartOfMonth, toStartOfYear, toStartOfMinute, toStartOfHour
* toTime,
* toMonday, toStartOfMonth, toStartOfYear, toStartOfMinute, toStartOfFiveMinute
* toStartOfHour, toTime,
* now
* TODO: makeDate, makeDateTime
*
@ -141,6 +141,15 @@ struct ToStartOfMinuteImpl
}
};
struct ToStartOfFiveMinuteImpl
{
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfFiveMinuteInaccurate(t); }
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
{
throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
struct ToStartOfHourImpl
{
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfHourInaccurate(t); }
@ -597,6 +606,7 @@ struct NameToStartOfMonth { static constexpr auto name = "toStartOfMonth"; };
struct NameToStartOfQuarter { static constexpr auto name = "toStartOfQuarter"; };
struct NameToStartOfYear { static constexpr auto name = "toStartOfYear"; };
struct NameToStartOfMinute { static constexpr auto name = "toStartOfMinute"; };
struct NameToStartOfFiveMinute { static constexpr auto name = "toStartOfFiveMinute"; };
struct NameToStartOfHour { static constexpr auto name = "toStartOfHour"; };
struct NameToTime { static constexpr auto name = "toTime"; };
struct NameToRelativeYearNum { static constexpr auto name = "toRelativeYearNum"; };
@ -620,6 +630,7 @@ typedef FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfMonthImpl, Nam
typedef FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfQuarterImpl, NameToStartOfQuarter> FunctionToStartOfQuarter;
typedef FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfYearImpl, NameToStartOfYear> FunctionToStartOfYear;
typedef FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfMinuteImpl, NameToStartOfMinute> FunctionToStartOfMinute;
typedef FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfFiveMinuteImpl, NameToStartOfFiveMinute> FunctionToStartOfFiveMinute;
typedef FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfHourImpl, NameToStartOfHour> FunctionToStartOfHour;
typedef FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToTimeImpl, NameToTime> FunctionToTime;

View File

@ -165,9 +165,8 @@ struct UnaryFunctionVectorized
template <typename T>
static void execute(const T * const src, Float64 * const dst)
{
const auto & result = Function(Vec2d(src[0], src[1]));
dst[0] = result[0];
dst[1] = result[1];
const auto result = Function(Vec2d(src[0], src[1]));
result.store(dst);
}
};
@ -432,9 +431,8 @@ struct BinaryFunctionVectorized
template <typename T1, typename T2>
static void execute(const T1 * const src_left, const T2 * const src_right, Float64 * const dst)
{
const auto & result = Function(Vec2d(src_left[0], src_left[1]), Vec2d(src_right[0], src_right[1]));
dst[0] = result[0];
dst[1] = result[1];
const auto result = Function(Vec2d(src_left[0], src_left[1]), Vec2d(src_right[0], src_right[1]));
result.store(dst);
}
};

View File

@ -1,10 +1,9 @@
#pragma once
#include <Poco/Net/DNS.h>
#include <Yandex/Revision.h>
#include <math.h>
#include <mutex>
#include <DB/Core/Defines.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
@ -22,11 +21,11 @@
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnReplicated.h>
#include <DB/Common/UnicodeBar.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Functions/IFunction.h>
#include <DB/Interpreters/ExpressionActions.h>
#include <statdaemons/ext/range.hpp>
#include <cmath>
namespace DB
{
@ -58,7 +57,7 @@ namespace DB
*
* bar(x, min, max, width) - рисует полосу из количества символов, пропорционального (x - min) и равного width при x == max.
*
* transform(x, from_array, to_array[, default]) - преобразовать x согласно переданному явным образом соответствию.
* version() - возвращает текущую версию сервера в строке.
*/
@ -112,6 +111,7 @@ public:
}
};
/// Получить имя хоста. (Оно - константа, вычисляется один раз за весь запрос.)
class FunctionHostName : public IFunction
{
@ -146,6 +146,7 @@ public:
}
};
class FunctionVisibleWidth : public IFunction
{
public:
@ -889,505 +890,33 @@ using FunctionIsFinite = FunctionNumericPredicate<IsFiniteImpl>;
using FunctionIsInfinite = FunctionNumericPredicate<IsInfiniteImpl>;
using FunctionIsNaN = FunctionNumericPredicate<IsNaNImpl>;
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2);
/** transform(x, [from...], [to...], default)
* - преобразует значения согласно явно указанному отображению.
*
* x - что преобразовывать.
* from - константный массив значений для преобразования.
* to - константный массив значений, в которые должны быть преобразованы значения из from.
* default - константа, какое значение использовать, если x не равен ни одному из значений во from.
* from и to - массивы одинаковых размеров.
*
* Типы:
* transform(T, Array(T), Array(U), U) -> U
*
* transform(x, [from...], [to...])
* - eсли default не указан, то для значений x, для которых нет соответствующего элемента во from, возвращается не изменённое значение x.
*
* Типы:
* transform(T, Array(T), Array(T)) -> T
*
* Замечание: реализация довольно громоздкая.
*/
class FunctionTransform : public IFunction
class FunctionVersion : public IFunction
{
public:
static constexpr auto name = "transform";
static IFunction * create(const Context &) { return new FunctionTransform; }
static constexpr auto name = "version";
static IFunction * create(const Context & context) { return new FunctionVersion; }
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
{
const auto args_size = arguments.size();
if (args_size != 3 && args_size != 4)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(args_size) + ", should be 3 or 4",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const IDataType * type_x = arguments[0].get();
if (!type_x->isNumeric() && !typeid_cast<const DataTypeString *>(type_x))
throw Exception("Unsupported type " + type_x->getName()
+ " of first argument of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * type_arr_from = typeid_cast<const DataTypeArray *>(arguments[1].get());
if (!type_arr_from)
throw Exception("Second argument of function " + getName()
+ ", must be array of source values to transform from.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto type_arr_from_nested = type_arr_from->getNestedType();
if ((type_x->isNumeric() != type_arr_from_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_from_nested.get())))
throw Exception("First argument and elements of array of second argument of function " + getName()
+ " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * type_arr_to = typeid_cast<const DataTypeArray *>(arguments[2].get());
if (!type_arr_to)
throw Exception("Third argument of function " + getName()
+ ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto type_arr_to_nested = type_arr_to->getNestedType();
if (args_size == 3)
{
if ((type_x->isNumeric() != type_arr_to_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
throw Exception("Function " + getName()
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_x->clone();
}
else
{
const IDataType * type_default = arguments[3].get();
if (!type_default->isNumeric() && !typeid_cast<const DataTypeString *>(type_default))
throw Exception("Unsupported type " + type_default->getName()
+ " of fourth argument (default value) of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if ((type_default->isNumeric() != type_arr_to_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_default) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
throw Exception("Function " + getName()
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (type_arr_to_nested->behavesAsNumber() && type_default->behavesAsNumber())
{
/// Берём наименьший общий тип для элементов массива значений to и для default-а.
return getSmallestCommonNumericType(*type_arr_to_nested, *type_default);
}
/// TODO Больше проверок.
return type_arr_to_nested->clone();
}
if (!arguments.empty())
throw Exception("Function " + getName() + " must be called without arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new DataTypeString;
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnConstArray * array_from = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[1]).column);
const ColumnConstArray * array_to = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[2]).column);
if (!array_from && !array_to)
throw Exception("Second and third arguments of function " + getName() + " must be constant arrays.", ErrorCodes::ILLEGAL_COLUMN);
prepare(array_from->getData(), array_to->getData(), block, arguments);
const auto in = block.getByPosition(arguments.front()).column.get();
if (in->isConst())
{
executeConst(block, arguments, result);
return;
}
auto column_result = block.getByPosition(result).type->createColumn();
auto out = column_result.get();
if (!executeNum<UInt8>(in, out)
&& !executeNum<UInt16>(in, out)
&& !executeNum<UInt32>(in, out)
&& !executeNum<UInt64>(in, out)
&& !executeNum<Int8>(in, out)
&& !executeNum<Int16>(in, out)
&& !executeNum<Int32>(in, out)
&& !executeNum<Int64>(in, out)
&& !executeNum<Float32>(in, out)
&& !executeNum<Float64>(in, out)
&& !executeString(in, out))
throw Exception(
"Illegal column " + in->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = column_result;
static const std::string version = getVersion();
block.getByPosition(result).column = new ColumnConstString(version.length(), version);
}
private:
void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result)
std::string getVersion() const
{
/// Составим блок из полноценных столбцов размера 1 и вычислим функцию как обычно.
Block tmp_block;
ColumnNumbers tmp_arguments;
tmp_block.insert(block.getByPosition(arguments[0]));
tmp_block.getByPosition(0).column = static_cast<IColumnConst *>(tmp_block.getByPosition(0).column->cloneResized(1).get())->convertToFullColumn();
tmp_arguments.push_back(0);
for (size_t i = 1; i < arguments.size(); ++i)
{
tmp_block.insert(block.getByPosition(arguments[i]));
tmp_arguments.push_back(i);
}
tmp_block.insert(block.getByPosition(result));
size_t tmp_result = arguments.size();
execute(tmp_block, tmp_arguments, tmp_result);
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(
block.rowsInFirstColumn(),
(*tmp_block.getByPosition(tmp_result).column)[0]);
}
template <typename T>
bool executeNum(const IColumn * in_untyped, IColumn * out_untyped)
{
if (const auto in = typeid_cast<const ColumnVector<T> *>(in_untyped))
{
if (default_value.isNull())
{
auto out = typeid_cast<ColumnVector<T> *>(out_untyped);
if (!out)
throw Exception(
"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName()
+ ", must be " + in->getName(),
ErrorCodes::ILLEGAL_COLUMN);
executeImplNumToNum<T>(in->getData(), out->getData());
}
else
{
if (!executeNumToNumWithDefault<T, UInt8>(in, out_untyped)
&& !executeNumToNumWithDefault<T, UInt16>(in, out_untyped)
&& !executeNumToNumWithDefault<T, UInt32>(in, out_untyped)
&& !executeNumToNumWithDefault<T, UInt64>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Int8>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Int16>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Int32>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Int64>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Float32>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Float64>(in, out_untyped)
&& !executeNumToString<T>(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return true;
}
return false;
}
bool executeString(const IColumn * in_untyped, IColumn * out_untyped)
{
if (const auto in = typeid_cast<const ColumnString *>(in_untyped))
{
if (!executeStringToNum<UInt8>(in, out_untyped)
&& !executeStringToNum<UInt16>(in, out_untyped)
&& !executeStringToNum<UInt32>(in, out_untyped)
&& !executeStringToNum<UInt64>(in, out_untyped)
&& !executeStringToNum<Int8>(in, out_untyped)
&& !executeStringToNum<Int16>(in, out_untyped)
&& !executeStringToNum<Int32>(in, out_untyped)
&& !executeStringToNum<Int64>(in, out_untyped)
&& !executeStringToNum<Float32>(in, out_untyped)
&& !executeStringToNum<Float64>(in, out_untyped)
&& !executeStringToString(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
return true;
}
return false;
}
template <typename T, typename U>
bool executeNumToNumWithDefault(const ColumnVector<T> * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
executeImplNumToNumWithDefault<T, U>(in->getData(), out->getData(), default_value.get<U>());
return true;
}
template <typename T>
bool executeNumToString(const ColumnVector<T> * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
const String & default_str = default_value.get<const String &>();
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
executeImplNumToStringWithDefault<T>(in->getData(), out->getChars(), out->getOffsets(), default_string_ref);
return true;
}
template <typename U>
bool executeStringToNum(const ColumnString * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
executeImplStringToNumWithDefault<U>(in->getChars(), in->getOffsets(), out->getData(), default_value.get<U>());
return true;
}
bool executeStringToString(const ColumnString * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
if (default_value.isNull())
executeImplStringToString<false>(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), {});
else
{
const String & default_str = default_value.get<const String &>();
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
executeImplStringToString<true>(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), default_string_ref);
}
return true;
}
template <typename T, typename U>
void executeImplNumToNumWithDefault(const PODArray<T> & src, PODArray<U> & dst, U dst_default)
{
const auto & table = *table_num_to_num;
size_t size = src.size();
dst.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i])); /// little endian.
else
dst[i] = dst_default;
}
}
template <typename T>
void executeImplNumToNum(const PODArray<T> & src, PODArray<T> & dst)
{
const auto & table = *table_num_to_num;
size_t size = src.size();
dst.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
else
dst[i] = src[i];
}
}
template <typename T>
void executeImplNumToStringWithDefault(const PODArray<T> & src,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
{
const auto & table = *table_num_to_string;
size_t size = src.size();
dst_offsets.resize(size);
ColumnString::Offset_t current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
StringRef ref = it != table.end() ? it->second : dst_default;
dst_data.resize(current_offset + ref.size);
memcpy(&dst_data[current_offset], ref.data, ref.size);
current_offset += ref.size;
dst_offsets[i] = current_offset;
}
}
template <typename U>
void executeImplStringToNumWithDefault(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
PODArray<U> & dst, U dst_default)
{
const auto & table = *table_string_to_num;
size_t size = src_offsets.size();
dst.resize(size);
ColumnString::Offset_t current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
StringRef ref{&src_data[current_offset], src_offsets[i] - current_offset};
current_offset = src_offsets[i];
auto it = table.find(ref);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
else
dst[i] = dst_default;
}
}
template <bool with_default>
void executeImplStringToString(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
{
const auto & table = *table_string_to_string;
size_t size = src_offsets.size();
dst_offsets.resize(size);
ColumnString::Offset_t current_src_offset = 0;
ColumnString::Offset_t current_dst_offset = 0;
for (size_t i = 0; i < size; ++i)
{
StringRef src_ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset};
current_src_offset = src_offsets[i];
auto it = table.find(src_ref);
StringRef dst_ref = it != table.end() ? it->second : (with_default ? dst_default : src_ref);
dst_data.resize(current_dst_offset + dst_ref.size);
memcpy(&dst_data[current_dst_offset], dst_ref.data, dst_ref.size);
current_dst_offset += dst_ref.size;
dst_offsets[i] = current_dst_offset;
}
}
/// Разные варианты хэш-таблиц для реализации отображения.
using NumToNum = HashMap<UInt64, UInt64, HashCRC32<UInt64>>;
using NumToString = HashMap<UInt64, StringRef, HashCRC32<UInt64>>; /// Везде StringRef-ы с завершающим нулём.
using StringToNum = HashMap<StringRef, UInt64>;
using StringToString = HashMap<StringRef, StringRef>;
std::unique_ptr<NumToNum> table_num_to_num;
std::unique_ptr<NumToString> table_num_to_string;
std::unique_ptr<StringToNum> table_string_to_num;
std::unique_ptr<StringToString> table_string_to_string;
Arena string_pool;
Field default_value; /// Null, если не задано.
bool prepared = false;
std::mutex mutex;
/// Может вызываться из разных потоков. Срабатывает только при первом вызове.
void prepare(const Array & from, const Array & to, Block & block, const ColumnNumbers & arguments)
{
if (prepared)
return;
const size_t size = from.size();
if (0 == size)
throw Exception("Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS);
std::lock_guard<std::mutex> lock(mutex);
if (prepared)
return;
if (from.size() != to.size())
throw Exception("Second and third arguments of function " + getName() + " must be arrays of same size", ErrorCodes::BAD_ARGUMENTS);
Array converted_to;
const Array * used_to = &to;
/// Задано ли значение по-умолчанию.
if (arguments.size() == 4)
{
const IColumnConst * default_col = dynamic_cast<const IColumnConst *>(&*block.getByPosition(arguments[3]).column);
if (!default_col)
throw Exception("Fourth argument of function " + getName() + " (default value) must be constant", ErrorCodes::ILLEGAL_COLUMN);
default_value = (*default_col)[0];
/// Нужно ли преобразовать элементы to и default_value к наименьшему общему типу, который является Float64?
if (default_value.getType() == Field::Types::Float64 && to[0].getType() != Field::Types::Float64)
{
converted_to.resize(to.size());
for (size_t i = 0, size = to.size(); i < size; ++i)
converted_to[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), to[i]);
used_to = &converted_to;
}
else if (default_value.getType() != Field::Types::Float64 && to[0].getType() == Field::Types::Float64)
{
default_value = apply_visitor(FieldVisitorConvertToNumber<Float64>(), default_value);
}
}
/// Замечание: не делается проверка дубликатов в массиве from.
if (from[0].getType() != Field::Types::String && to[0].getType() != Field::Types::String)
{
table_num_to_num.reset(new NumToNum);
auto & table = *table_num_to_num;
for (size_t i = 0; i < size; ++i)
table[from[i].get<UInt64>()] = (*used_to)[i].get<UInt64>();
}
else if (from[0].getType() != Field::Types::String && to[0].getType() == Field::Types::String)
{
table_num_to_string.reset(new NumToString);
auto & table = *table_num_to_string;
for (size_t i = 0; i < size; ++i)
{
const String & str_to = to[i].get<const String &>();
StringRef ref{string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
table[from[i].get<UInt64>()] = ref;
}
}
else if (from[0].getType() == Field::Types::String && to[0].getType() != Field::Types::String)
{
table_string_to_num.reset(new StringToNum);
auto & table = *table_string_to_num;
for (size_t i = 0; i < size; ++i)
{
const String & str_from = from[i].get<const String &>();
StringRef ref{string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
table[ref] = (*used_to)[i].get<UInt64>();
}
}
else if (from[0].getType() == Field::Types::String && to[0].getType() == Field::Types::String)
{
table_string_to_string.reset(new StringToString);
auto & table = *table_string_to_string;
for (size_t i = 0; i < size; ++i)
{
const String & str_from = from[i].get<const String &>();
const String & str_to = to[i].get<const String &>();
StringRef ref_from{string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
StringRef ref_to{string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
table[ref_from] = ref_to;
}
}
prepared = true;
std::ostringstream os;
os << DBMS_VERSION_MAJOR << "." << DBMS_VERSION_MINOR << "." << Revision::get();
return os.str();
}
};

File diff suppressed because it is too large Load Diff

View File

@ -14,6 +14,9 @@
#include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp>
#include <emmintrin.h>
#include <nmmintrin.h>
namespace DB
{
@ -234,6 +237,255 @@ private:
}
};
template <char not_case_lower_bound, char not_case_upper_bound>
struct LowerUpperImplVectorized
{
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,
ColumnString::Chars_t & res_data, ColumnString::Offsets_t & res_offsets)
{
res_data.resize(data.size());
res_offsets.assign(offsets);
array(data.data(), data.data() + data.size(), res_data.data());
}
static void vector_fixed(const ColumnString::Chars_t & data, size_t n,
ColumnString::Chars_t & res_data)
{
res_data.resize(data.size());
array(data.data(), data.data() + data.size(), res_data.data());
}
static void constant(const std::string & data, std::string & res_data)
{
res_data.resize(data.size());
array(reinterpret_cast<const UInt8 *>(data.data()), reinterpret_cast<const UInt8 *>(data.data() + data.size()),
reinterpret_cast<UInt8 *>(&res_data[0]));
}
private:
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
{
const auto bytes_sse = sizeof(__m128i);
const auto src_end_sse = src_end - (src_end - src) % bytes_sse;
const auto flip_case_mask = 'A' ^ 'a';
const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1);
const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1);
const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask);
for (; src < src_end_sse; src += bytes_sse, dst += bytes_sse)
{
/// load 16 sequential 8-bit characters
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(src));
/// find which 8-bit sequences belong to range [case_lower_bound, case_upper_bound]
const auto is_not_case = _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound),
_mm_cmplt_epi8(chars, v_not_case_upper_bound));
/// keep `flip_case_mask` only where necessary, zero out elsewhere
const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case);
/// flip case by applying calculated mask
const auto cased_chars = _mm_xor_si128(chars, xor_mask);
/// store result back to destination
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars);
}
for (; src < src_end; ++src, ++dst)
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
*dst = *src ^ flip_case_mask;
else
*dst = *src;
}
};
/// xor or do nothing
template <bool> UInt8 xor_or_identity(const UInt8 c, const int mask) { return c ^ mask; };
template <> inline UInt8 xor_or_identity<false>(const UInt8 c, const int) { return c; }
/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array
template <bool to_lower>
inline void UTF8CyrillicToCase(const UInt8 * & src, const UInt8 * const src_end, UInt8 * & dst)
{
if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
{
/// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<to_lower>(*src++, 0x10);
}
else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
{
/// ѐёђѓєѕіїјљњћќѝўџ
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<!to_lower>(*src++, 0x10);
}
else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
{
/// А
*dst++ = *src++;
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
}
else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu))
{
/// а-п
*dst++ = *src++;
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
}
else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu))
{
/// Р
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
}
else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
{
/// р
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
}
};
template <char not_case_lower_bound, char not_case_upper_bound,
int to_case(int), void cyrillic_to_case(const UInt8 * &, const UInt8 *, UInt8 * &)>
struct LowerUpperUTF8ImplVectorized
{
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,
ColumnString::Chars_t & res_data, ColumnString::Offsets_t & res_offsets)
{
res_data.resize(data.size());
res_offsets.assign(offsets);
array(data.data(), data.data() + data.size(), res_data.data());
}
static void vector_fixed(const ColumnString::Chars_t & data, size_t n,
ColumnString::Chars_t & res_data)
{
res_data.resize(data.size());
array(data.data(), data.data() + data.size(), res_data.data());
}
static void constant(const std::string & data, std::string & res_data)
{
res_data.resize(data.size());
array(reinterpret_cast<const UInt8 *>(data.data()), reinterpret_cast<const UInt8 *>(data.data() + data.size()),
reinterpret_cast<UInt8 *>(&res_data[0]));
}
private:
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
{
static const Poco::UTF8Encoding utf8;
const auto bytes_sse = sizeof(__m128i);
auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse;
const auto flip_case_mask = 'A' ^ 'a';
const auto ascii_upper_bound = '\x7f';
/// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f)
const auto v_zero = _mm_setzero_si128();
const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1);
const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1);
const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask);
while (src < src_end_sse)
{
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(src));
/// check for ASCII
const auto is_not_ascii = _mm_cmplt_epi8(chars, v_zero);
const auto mask_is_not_ascii = _mm_movemask_epi8(is_not_ascii);
/// ASCII
if (mask_is_not_ascii == 0)
{
const auto is_not_case = _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound),
_mm_cmplt_epi8(chars, v_not_case_upper_bound));
const auto mask_is_not_case = _mm_movemask_epi8(is_not_case);
/// everything in correct case ASCII
if (mask_is_not_case == 0)
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), chars);
else
{
/// ASCII in mixed case
/// keep `flip_case_mask` only where necessary, zero out elsewhere
const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case);
/// flip case by applying calculated mask
const auto cased_chars = _mm_xor_si128(chars, xor_mask);
/// store result back to destination
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars);
}
src += bytes_sse, dst += bytes_sse;
}
else
{
/// UTF-8
const auto expected_end = src + bytes_sse;
while (src < expected_end)
{
if (src[0] <= ascii_upper_bound)
{
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
*dst++ = *src++ ^ flip_case_mask;
else
*dst++ = *src++;
}
else if (src + 1 < src_end &&
((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) ||
(src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu))))
{
cyrillic_to_case(src, src_end, dst);
}
else if (src + 1 < src_end && src[0] == 0xC2u)
{
/// Пунктуация U+0080 - U+00BF, UTF-8: C2 80 - C2 BF
*dst++ = *src++;
*dst++ = *src++;
}
else if (src + 2 < src_end && src[0] == 0xE2u)
{
/// Символы U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF
*dst++ = *src++;
*dst++ = *src++;
*dst++ = *src++;
}
else
{
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
src += chars, dst += chars;
else
++src, ++dst;
}
}
/// adjust src_end_sse by pushing it forward or backward
const auto diff = src - expected_end;
if (diff != 0)
{
if (src_end_sse + diff < src_end)
src_end_sse += diff;
else
src_end_sse -= bytes_sse - diff;
}
}
}
/// handle remaining symbols
while (src < src_end)
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
src += chars, dst += chars;
else
++src, ++dst;
}
};
/** Если строка содержит текст в кодировке UTF-8 - перевести его в нижний (верхний) регистр.
* Замечание: предполагается, что после перевода символа в другой регистр,
@ -1424,6 +1676,11 @@ struct NameReverseUTF8 { static constexpr auto name = "reverseUTF8"; };
struct NameSubstring { static constexpr auto name = "substring"; };
struct NameSubstringUTF8 { static constexpr auto name = "substringUTF8"; };
struct NameSSELower { static constexpr auto name = "sse_lower"; };
struct NameSSEUpper { static constexpr auto name = "sse_upper"; };
struct NameSSELowerUTF8 { static constexpr auto name = "sse_lowerUTF8"; };
struct NameSSEUpperUTF8 { static constexpr auto name = "sse_upperUTF8"; };
typedef FunctionStringOrArrayToT<EmptyImpl<false>, NameEmpty, UInt8> FunctionEmpty;
typedef FunctionStringOrArrayToT<EmptyImpl<true>, NameNotEmpty, UInt8> FunctionNotEmpty;
typedef FunctionStringOrArrayToT<LengthImpl, NameLength, UInt64> FunctionLength;
@ -1437,5 +1694,14 @@ typedef FunctionStringToString<ReverseUTF8Impl, NameReverseUTF8> FunctionReve
typedef FunctionStringNumNumToString<SubstringImpl, NameSubstring> FunctionSubstring;
typedef FunctionStringNumNumToString<SubstringUTF8Impl, NameSubstringUTF8> FunctionSubstringUTF8;
using FunctionSSELower = FunctionStringToString<LowerUpperImplVectorized<'A', 'Z'>, NameSSELower>;
using FunctionSSEUpper = FunctionStringToString<LowerUpperImplVectorized<'a', 'z'>, NameSSEUpper>;
using FunctionSSELowerUTF8 = FunctionStringToString<
LowerUpperUTF8ImplVectorized<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>,
NameSSELowerUTF8>;
using FunctionSSEUpperUTF8 = FunctionStringToString<
LowerUpperUTF8ImplVectorized<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>,
NameSSEUpperUTF8>;
}

View File

@ -0,0 +1,810 @@
#pragma once
#include <mutex>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Common/Arena.h>
#include <DB/Core/StringRef.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Functions/IFunction.h>
namespace DB
{
/** transform(x, from_array, to_array[, default]) - преобразовать x согласно переданному явным образом соответствию.
*/
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2);
/** transform(x, [from...], [to...], default)
* - преобразует значения согласно явно указанному отображению.
*
* x - что преобразовывать.
* from - константный массив значений для преобразования.
* to - константный массив значений, в которые должны быть преобразованы значения из from.
* default - какое значение использовать, если x не равен ни одному из значений во from.
* from и to - массивы одинаковых размеров.
*
* Типы:
* transform(T, Array(T), Array(U), U) -> U
*
* transform(x, [from...], [to...])
* - eсли default не указан, то для значений x, для которых нет соответствующего элемента во from, возвращается не изменённое значение x.
*
* Типы:
* transform(T, Array(T), Array(T)) -> T
*
* Замечание: реализация довольно громоздкая.
*/
class FunctionTransform : public IFunction
{
public:
static constexpr auto name = "transform";
static IFunction * create(const Context &) { return new FunctionTransform; }
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
{
const auto args_size = arguments.size();
if (args_size != 3 && args_size != 4)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(args_size) + ", should be 3 or 4",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const IDataType * type_x = arguments[0].get();
if (!type_x->isNumeric() && !typeid_cast<const DataTypeString *>(type_x))
throw Exception("Unsupported type " + type_x->getName()
+ " of first argument of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * type_arr_from = typeid_cast<const DataTypeArray *>(arguments[1].get());
if (!type_arr_from)
throw Exception("Second argument of function " + getName()
+ ", must be array of source values to transform from.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto type_arr_from_nested = type_arr_from->getNestedType();
if ((type_x->isNumeric() != type_arr_from_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_from_nested.get())))
throw Exception("First argument and elements of array of second argument of function " + getName()
+ " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * type_arr_to = typeid_cast<const DataTypeArray *>(arguments[2].get());
if (!type_arr_to)
throw Exception("Third argument of function " + getName()
+ ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto type_arr_to_nested = type_arr_to->getNestedType();
if (args_size == 3)
{
if ((type_x->isNumeric() != type_arr_to_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
throw Exception("Function " + getName()
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_x->clone();
}
else
{
const IDataType * type_default = arguments[3].get();
if (!type_default->isNumeric() && !typeid_cast<const DataTypeString *>(type_default))
throw Exception("Unsupported type " + type_default->getName()
+ " of fourth argument (default value) of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if ((type_default->isNumeric() != type_arr_to_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_default) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
throw Exception("Function " + getName()
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (type_arr_to_nested->behavesAsNumber() && type_default->behavesAsNumber())
{
/// Берём наименьший общий тип для элементов массива значений to и для default-а.
return getSmallestCommonNumericType(*type_arr_to_nested, *type_default);
}
/// TODO Больше проверок.
return type_arr_to_nested->clone();
}
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const ColumnConstArray * array_from = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[1]).column);
const ColumnConstArray * array_to = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[2]).column);
if (!array_from && !array_to)
throw Exception("Second and third arguments of function " + getName() + " must be constant arrays.", ErrorCodes::ILLEGAL_COLUMN);
prepare(array_from->getData(), array_to->getData(), block, arguments);
const auto in = block.getByPosition(arguments.front()).column.get();
if (in->isConst())
{
executeConst(block, arguments, result);
return;
}
const IColumn * default_column = nullptr;
if (arguments.size() == 4)
default_column = block.getByPosition(arguments[3]).column.get();
auto column_result = block.getByPosition(result).type->createColumn();
auto out = column_result.get();
if (!executeNum<UInt8>(in, out, default_column)
&& !executeNum<UInt16>(in, out, default_column)
&& !executeNum<UInt32>(in, out, default_column)
&& !executeNum<UInt64>(in, out, default_column)
&& !executeNum<Int8>(in, out, default_column)
&& !executeNum<Int16>(in, out, default_column)
&& !executeNum<Int32>(in, out, default_column)
&& !executeNum<Int64>(in, out, default_column)
&& !executeNum<Float32>(in, out, default_column)
&& !executeNum<Float64>(in, out, default_column)
&& !executeString(in, out, default_column))
throw Exception(
"Illegal column " + in->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = column_result;
}
private:
void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result)
{
/// Составим блок из полноценных столбцов размера 1 и вычислим функцию как обычно.
Block tmp_block;
ColumnNumbers tmp_arguments;
tmp_block.insert(block.getByPosition(arguments[0]));
tmp_block.getByPosition(0).column = static_cast<IColumnConst *>(tmp_block.getByPosition(0).column->cloneResized(1).get())->convertToFullColumn();
tmp_arguments.push_back(0);
for (size_t i = 1; i < arguments.size(); ++i)
{
tmp_block.insert(block.getByPosition(arguments[i]));
tmp_arguments.push_back(i);
}
tmp_block.insert(block.getByPosition(result));
size_t tmp_result = arguments.size();
execute(tmp_block, tmp_arguments, tmp_result);
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(
block.rowsInFirstColumn(),
(*tmp_block.getByPosition(tmp_result).column)[0]);
}
template <typename T>
bool executeNum(const IColumn * in_untyped, IColumn * out_untyped, const IColumn * default_untyped)
{
if (const auto in = typeid_cast<const ColumnVector<T> *>(in_untyped))
{
if (!default_untyped)
{
auto out = typeid_cast<ColumnVector<T> *>(out_untyped);
if (!out)
throw Exception(
"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName()
+ ", must be " + in->getName(),
ErrorCodes::ILLEGAL_COLUMN);
executeImplNumToNum<T>(in->getData(), out->getData());
}
else if (default_untyped->isConst())
{
if (!executeNumToNumWithConstDefault<T, UInt8>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, UInt16>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, UInt32>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, UInt64>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Int8>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Int16>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Int32>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Int64>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Float32>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Float64>(in, out_untyped)
&& !executeNumToStringWithConstDefault<T>(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
else
{
if (!executeNumToNumWithNonConstDefault<T, UInt8>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, UInt16>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, UInt32>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, UInt64>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Int8>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Int16>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Int32>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Int64>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Float32>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Float64>(in, out_untyped, default_untyped)
&& !executeNumToStringWithNonConstDefault<T>(in, out_untyped, default_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return true;
}
return false;
}
bool executeString(const IColumn * in_untyped, IColumn * out_untyped, const IColumn * default_untyped)
{
if (const auto in = typeid_cast<const ColumnString *>(in_untyped))
{
if (!default_untyped)
{
if (!executeStringToString(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
else if (default_untyped->isConst())
{
if (!executeStringToNumWithConstDefault<UInt8>(in, out_untyped)
&& !executeStringToNumWithConstDefault<UInt16>(in, out_untyped)
&& !executeStringToNumWithConstDefault<UInt32>(in, out_untyped)
&& !executeStringToNumWithConstDefault<UInt64>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Int8>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Int16>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Int32>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Int64>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Float32>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Float64>(in, out_untyped)
&& !executeStringToStringWithConstDefault(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
else
{
if (!executeStringToNumWithNonConstDefault<UInt8>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<UInt16>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<UInt32>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<UInt64>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Int8>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Int16>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Int32>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Int64>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Float32>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Float64>(in, out_untyped, default_untyped)
&& !executeStringToStringWithNonConstDefault(in, out_untyped, default_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return true;
}
return false;
}
template <typename T, typename U>
bool executeNumToNumWithConstDefault(const ColumnVector<T> * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
executeImplNumToNumWithConstDefault<T, U>(in->getData(), out->getData(), const_default_value.get<U>());
return true;
}
template <typename T, typename U>
bool executeNumToNumWithNonConstDefault(const ColumnVector<T> * in, IColumn * out_untyped, const IColumn * default_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
if (!executeNumToNumWithNonConstDefault2<T, U, UInt8>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, UInt16>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, UInt32>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, UInt64>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Int8>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Int16>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Int32>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Int64>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Float32>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Float64>(in, out, default_untyped))
throw Exception(
"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
return true;
}
template <typename T, typename U, typename V>
bool executeNumToNumWithNonConstDefault2(const ColumnVector<T> * in, ColumnVector<U> * out, const IColumn * default_untyped)
{
auto col_default = typeid_cast<const ColumnVector<V> *>(default_untyped);
if (!col_default)
return false;
executeImplNumToNumWithNonConstDefault<T, U, V>(in->getData(), out->getData(), col_default->getData());
return true;
}
template <typename T>
bool executeNumToStringWithConstDefault(const ColumnVector<T> * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
const String & default_str = const_default_value.get<const String &>();
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
executeImplNumToStringWithConstDefault<T>(in->getData(), out->getChars(), out->getOffsets(), default_string_ref);
return true;
}
template <typename T>
bool executeNumToStringWithNonConstDefault(const ColumnVector<T> * in, IColumn * out_untyped, const IColumn * default_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
auto default_col = typeid_cast<const ColumnString *>(default_untyped);
if (!default_col)
throw Exception("Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
executeImplNumToStringWithNonConstDefault<T>(
in->getData(),
out->getChars(), out->getOffsets(),
default_col->getChars(), default_col->getOffsets());
return true;
}
template <typename U>
bool executeStringToNumWithConstDefault(const ColumnString * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
executeImplStringToNumWithConstDefault<U>(in->getChars(), in->getOffsets(), out->getData(), const_default_value.get<U>());
return true;
}
template <typename U>
bool executeStringToNumWithNonConstDefault(const ColumnString * in, IColumn * out_untyped, const IColumn * default_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
if (!executeStringToNumWithNonConstDefault2<U, UInt8>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, UInt16>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, UInt32>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, UInt64>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Int8>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Int16>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Int32>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Int64>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Float32>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Float64>(in, out, default_untyped))
throw Exception(
"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
return true;
}
template <typename U, typename V>
bool executeStringToNumWithNonConstDefault2(const ColumnString * in, ColumnVector<U> * out, const IColumn * default_untyped)
{
auto col_default = typeid_cast<const ColumnVector<V> *>(default_untyped);
if (!col_default)
return false;
executeImplStringToNumWithNonConstDefault<U, V>(in->getChars(), in->getOffsets(), out->getData(), col_default->getData());
return true;
}
bool executeStringToString(const ColumnString * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
executeImplStringToString(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets());
return true;
}
bool executeStringToStringWithConstDefault(const ColumnString * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
const String & default_str = const_default_value.get<const String &>();
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
executeImplStringToStringWithConstDefault(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), default_string_ref);
return true;
}
bool executeStringToStringWithNonConstDefault(const ColumnString * in, IColumn * out_untyped, const IColumn * default_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
auto default_col = typeid_cast<const ColumnString *>(default_untyped);
if (!default_col)
throw Exception("Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
executeImplStringToStringWithNonConstDefault(
in->getChars(), in->getOffsets(),
out->getChars(), out->getOffsets(),
default_col->getChars(), default_col->getOffsets());
return true;
}
template <typename T, typename U>
void executeImplNumToNumWithConstDefault(const PODArray<T> & src, PODArray<U> & dst, U dst_default)
{
const auto & table = *table_num_to_num;
size_t size = src.size();
dst.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i])); /// little endian.
else
dst[i] = dst_default;
}
}
template <typename T, typename U, typename V>
void executeImplNumToNumWithNonConstDefault(const PODArray<T> & src, PODArray<U> & dst, const PODArray<V> & dst_default)
{
const auto & table = *table_num_to_num;
size_t size = src.size();
dst.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i])); /// little endian.
else
dst[i] = dst_default[i];
}
}
template <typename T>
void executeImplNumToNum(const PODArray<T> & src, PODArray<T> & dst)
{
const auto & table = *table_num_to_num;
size_t size = src.size();
dst.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
else
dst[i] = src[i];
}
}
template <typename T>
void executeImplNumToStringWithConstDefault(const PODArray<T> & src,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
{
const auto & table = *table_num_to_string;
size_t size = src.size();
dst_offsets.resize(size);
ColumnString::Offset_t current_dst_offset = 0;
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
StringRef ref = it != table.end() ? it->second : dst_default;
dst_data.resize(current_dst_offset + ref.size);
memcpy(&dst_data[current_dst_offset], ref.data, ref.size);
current_dst_offset += ref.size;
dst_offsets[i] = current_dst_offset;
}
}
template <typename T>
void executeImplNumToStringWithNonConstDefault(const PODArray<T> & src,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets,
const ColumnString::Chars_t & dst_default_data, const ColumnString::Offsets_t & dst_default_offsets)
{
const auto & table = *table_num_to_string;
size_t size = src.size();
dst_offsets.resize(size);
ColumnString::Offset_t current_dst_offset = 0;
ColumnString::Offset_t current_dst_default_offset = 0;
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
StringRef ref;
if (it != table.end())
ref = it->second;
else
{
ref.data = reinterpret_cast<const char *>(&dst_default_data[current_dst_default_offset]);
ref.size = dst_default_offsets[i] - current_dst_default_offset;
}
dst_data.resize(current_dst_offset + ref.size);
memcpy(&dst_data[current_dst_offset], ref.data, ref.size);
current_dst_offset += ref.size;
current_dst_default_offset = dst_default_offsets[i];
dst_offsets[i] = current_dst_offset;
}
}
template <typename U>
void executeImplStringToNumWithConstDefault(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
PODArray<U> & dst, U dst_default)
{
const auto & table = *table_string_to_num;
size_t size = src_offsets.size();
dst.resize(size);
ColumnString::Offset_t current_src_offset = 0;
for (size_t i = 0; i < size; ++i)
{
StringRef ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset};
current_src_offset = src_offsets[i];
auto it = table.find(ref);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
else
dst[i] = dst_default;
}
}
template <typename U, typename V>
void executeImplStringToNumWithNonConstDefault(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
PODArray<U> & dst, const PODArray<V> & dst_default)
{
const auto & table = *table_string_to_num;
size_t size = src_offsets.size();
dst.resize(size);
ColumnString::Offset_t current_src_offset = 0;
for (size_t i = 0; i < size; ++i)
{
StringRef ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset};
current_src_offset = src_offsets[i];
auto it = table.find(ref);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
else
dst[i] = dst_default[i];
}
}
template <bool with_default>
void executeImplStringToStringWithOrWithoutConstDefault(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
{
const auto & table = *table_string_to_string;
size_t size = src_offsets.size();
dst_offsets.resize(size);
ColumnString::Offset_t current_src_offset = 0;
ColumnString::Offset_t current_dst_offset = 0;
for (size_t i = 0; i < size; ++i)
{
StringRef src_ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset};
current_src_offset = src_offsets[i];
auto it = table.find(src_ref);
StringRef dst_ref = it != table.end() ? it->second : (with_default ? dst_default : src_ref);
dst_data.resize(current_dst_offset + dst_ref.size);
memcpy(&dst_data[current_dst_offset], dst_ref.data, dst_ref.size);
current_dst_offset += dst_ref.size;
dst_offsets[i] = current_dst_offset;
}
}
void executeImplStringToString(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets)
{
executeImplStringToStringWithOrWithoutConstDefault<false>(src_data, src_offsets, dst_data, dst_offsets, {});
}
void executeImplStringToStringWithConstDefault(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
{
executeImplStringToStringWithOrWithoutConstDefault<true>(src_data, src_offsets, dst_data, dst_offsets, dst_default);
}
void executeImplStringToStringWithNonConstDefault(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets,
const ColumnString::Chars_t & dst_default_data, const ColumnString::Offsets_t & dst_default_offsets)
{
const auto & table = *table_string_to_string;
size_t size = src_offsets.size();
dst_offsets.resize(size);
ColumnString::Offset_t current_src_offset = 0;
ColumnString::Offset_t current_dst_offset = 0;
ColumnString::Offset_t current_dst_default_offset = 0;
for (size_t i = 0; i < size; ++i)
{
StringRef src_ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset};
current_src_offset = src_offsets[i];
auto it = table.find(src_ref);
StringRef dst_ref;
if (it != table.end())
dst_ref = it->second;
else
{
dst_ref.data = reinterpret_cast<const char *>(&dst_default_data[current_dst_default_offset]);
dst_ref.size = dst_default_offsets[i] - current_dst_default_offset;
}
dst_data.resize(current_dst_offset + dst_ref.size);
memcpy(&dst_data[current_dst_offset], dst_ref.data, dst_ref.size);
current_dst_offset += dst_ref.size;
current_dst_default_offset = dst_default_offsets[i];
dst_offsets[i] = current_dst_offset;
}
}
/// Разные варианты хэш-таблиц для реализации отображения.
using NumToNum = HashMap<UInt64, UInt64, HashCRC32<UInt64>>;
using NumToString = HashMap<UInt64, StringRef, HashCRC32<UInt64>>; /// Везде StringRef-ы с завершающим нулём.
using StringToNum = HashMap<StringRef, UInt64, StringRefHash>;
using StringToString = HashMap<StringRef, StringRef, StringRefHash>;
std::unique_ptr<NumToNum> table_num_to_num;
std::unique_ptr<NumToString> table_num_to_string;
std::unique_ptr<StringToNum> table_string_to_num;
std::unique_ptr<StringToString> table_string_to_string;
Arena string_pool;
Field const_default_value; /// Null, если не задано.
bool prepared = false;
std::mutex mutex;
/// Может вызываться из разных потоков. Срабатывает только при первом вызове.
void prepare(const Array & from, const Array & to, Block & block, const ColumnNumbers & arguments)
{
if (prepared)
return;
const size_t size = from.size();
if (0 == size)
throw Exception("Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS);
std::lock_guard<std::mutex> lock(mutex);
if (prepared)
return;
if (from.size() != to.size())
throw Exception("Second and third arguments of function " + getName() + " must be arrays of same size", ErrorCodes::BAD_ARGUMENTS);
Array converted_to;
const Array * used_to = &to;
/// Задано ли значение по-умолчанию.
if (arguments.size() == 4)
{
const IColumn * default_col = block.getByPosition(arguments[3]).column.get();
const IColumnConst * const_default_col = dynamic_cast<const IColumnConst *>(default_col);
if (const_default_col)
const_default_value = (*const_default_col)[0];
/// Нужно ли преобразовать элементы to и default_value к наименьшему общему типу, который является Float64?
bool default_col_is_float =
typeid_cast<const ColumnFloat32 *>(default_col)
|| typeid_cast<const ColumnFloat64 *>(default_col)
|| typeid_cast<const ColumnConstFloat32 *>(default_col)
|| typeid_cast<const ColumnConstFloat64 *>(default_col);
bool to_is_float = to[0].getType() == Field::Types::Float64;
if (default_col_is_float && !to_is_float)
{
converted_to.resize(to.size());
for (size_t i = 0, size = to.size(); i < size; ++i)
converted_to[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), to[i]);
used_to = &converted_to;
}
else if (!default_col_is_float && to_is_float)
{
if (const_default_col)
const_default_value = apply_visitor(FieldVisitorConvertToNumber<Float64>(), const_default_value);
}
}
/// Замечание: не делается проверка дубликатов в массиве from.
if (from[0].getType() != Field::Types::String && to[0].getType() != Field::Types::String)
{
table_num_to_num.reset(new NumToNum);
auto & table = *table_num_to_num;
for (size_t i = 0; i < size; ++i)
table[from[i].get<UInt64>()] = (*used_to)[i].get<UInt64>();
}
else if (from[0].getType() != Field::Types::String && to[0].getType() == Field::Types::String)
{
table_num_to_string.reset(new NumToString);
auto & table = *table_num_to_string;
for (size_t i = 0; i < size; ++i)
{
const String & str_to = to[i].get<const String &>();
StringRef ref{string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
table[from[i].get<UInt64>()] = ref;
}
}
else if (from[0].getType() == Field::Types::String && to[0].getType() != Field::Types::String)
{
table_string_to_num.reset(new StringToNum);
auto & table = *table_string_to_num;
for (size_t i = 0; i < size; ++i)
{
const String & str_from = from[i].get<const String &>();
StringRef ref{string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
table[ref] = (*used_to)[i].get<UInt64>();
}
}
else if (from[0].getType() == Field::Types::String && to[0].getType() == Field::Types::String)
{
table_string_to_string.reset(new StringToString);
auto & table = *table_string_to_string;
for (size_t i = 0; i < size; ++i)
{
const String & str_from = from[i].get<const String &>();
const String & str_to = to[i].get<const String &>();
StringRef ref_from{string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
StringRef ref_to{string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
table[ref_from] = ref_to;
}
}
prepared = true;
}
};
}

View File

@ -14,10 +14,10 @@ namespace DB
/** Метод сжатия */
enum class CompressionMethod
{
QuickLZ,
LZ4,
LZ4HC, /// Формат такой же, как у LZ4. Разница только при сжатии.
ZSTD, /// Экспериментальный алгоритм: https://github.com/Cyan4973/zstd
QuickLZ = 0,
LZ4 = 1,
LZ4HC = 2, /// Формат такой же, как у LZ4. Разница только при сжатии.
ZSTD = 3, /// Экспериментальный алгоритм: https://github.com/Cyan4973/zstd
};
/** Формат сжатого блока следующий:

View File

@ -2,13 +2,13 @@
#include <map>
#include <DB/Interpreters/Settings.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/Client/ConnectionPool.h>
#include <DB/Client/ConnectionPoolWithFailover.h>
#include <Poco/Net/SocketAddress.h>
namespace DB
{
/// Cluster содержит пулы соединений до каждого из узлов
/// С локальными узлами соединение не устанавливается, а выполяется запрос напрямую.
/// Поэтому храним только количество локальных узлов
@ -16,10 +16,10 @@ namespace DB
class Cluster : private boost::noncopyable
{
public:
Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, const String & cluster_name);
Cluster(const Settings & settings, const String & cluster_name);
/// Построить кластер по именам шардов и реплик. Локальные обрабатываются так же как удаленные.
Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, std::vector<std::vector<String>> names,
Cluster(const Settings & settings, std::vector<std::vector<String>> names,
const String & username, const String & password);
/// количество узлов clickhouse сервера, расположенных локально
@ -62,8 +62,9 @@ public:
* </replica>
* </shard>
*/
Poco::Net::SocketAddress host_port;
Poco::Net::SocketAddress resolved_address;
String host_name;
UInt16 port;
String user;
String password;
UInt32 replica_num;
@ -98,8 +99,7 @@ struct Clusters
typedef std::map<String, Cluster> Impl;
Impl impl;
Clusters(const Settings & settings, const DataTypeFactory & data_type_factory,
const String & config_name = "remote_servers");
Clusters(const Settings & settings, const String & config_name = "remote_servers");
};
}

View File

@ -160,7 +160,6 @@ public:
const TableFunctionFactory & getTableFunctionFactory() const;
const AggregateFunctionFactory & getAggregateFunctionFactory() const;
const DataTypeFactory & getDataTypeFactory() const;
const FormatFactory & getFormatFactory() const;
const Dictionaries & getDictionaries() const;
const ExternalDictionaries & getExternalDictionaries() const;

View File

@ -22,6 +22,7 @@ struct SubqueryForSet
{
/// Источник - получен с помощью InterpreterSelectQuery подзапроса.
BlockInputStreamPtr source;
Block source_sample;
/// Если задано - создать из результата Set.
SetPtr set;

View File

@ -43,10 +43,14 @@ private:
mutable std::mutex dictionaries_mutex;
using dictionary_ptr_t = std::shared_ptr<MultiVersion<IDictionary>>;
using dictionary_origin_pair_t = std::pair<dictionary_ptr_t, std::string>;
std::unordered_map<std::string, dictionary_origin_pair_t> dictionaries;
/// exception pointers for notifying user about failures on dictionary creation
std::unordered_map<std::string, std::exception_ptr> stored_exceptions;
struct dictionary_info final
{
dictionary_ptr_t dict;
std::string origin;
std::exception_ptr exception;
};
std::unordered_map<std::string, dictionary_info> dictionaries;
std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
std::mt19937_64 rnd_engine{getSeed()};
@ -95,24 +99,7 @@ public:
reloading_thread.join();
}
MultiVersion<IDictionary>::Version getDictionary(const std::string & name) const
{
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
const auto it = dictionaries.find(name);
if (it == std::end(dictionaries))
{
const auto exception_it = stored_exceptions.find(name);
if (exception_it != std::end(stored_exceptions))
std::rethrow_exception(exception_it->second);
else
throw Exception{
"No such dictionary: " + name,
ErrorCodes::BAD_ARGUMENTS
};
}
return it->second.first->get();
}
MultiVersion<IDictionary>::Version getDictionary(const std::string & name) const;
};
}

View File

@ -77,7 +77,7 @@ private:
Context context;
static void parseAlter(const ASTAlterQuery::ParameterContainer & params, const DataTypeFactory & data_type_factory,
static void parseAlter(const ASTAlterQuery::ParameterContainer & params,
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands);
};

View File

@ -74,6 +74,11 @@ public:
bool empty() { return type == Type::EMPTY; }
/** Передать информацию о структуре блока.
* Следует обязательно вызвать до вызовов insertFromBlock.
*/
void setSampleBlock(const Block & block);
/** Добавить в отображение для соединения блок "правой" таблицы.
* Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
*/
@ -217,6 +222,8 @@ private:
bool keys_fit_128_bits;
Sizes key_sizes;
Block sample_block;
Logger * log;
/// Ограничения на максимальный размер множества

View File

@ -72,13 +72,14 @@ struct QuotaForInterval
{
time_t rounded_time;
size_t duration;
time_t offset; /// Смещение интервала, для рандомизации.
QuotaValues max;
QuotaValues used;
QuotaForInterval() : rounded_time() {}
QuotaForInterval(time_t duration_) : duration(duration_) {}
void initFromConfig(const String & config_elem, time_t duration_, Poco::Util::AbstractConfiguration & config);
void initFromConfig(const String & config_elem, time_t duration_, time_t offset_, Poco::Util::AbstractConfiguration & config);
/// Увеличить соответствующее значение.
void addQuery(time_t current_time, const String & quota_name);
@ -131,7 +132,7 @@ public:
return cont.empty();
}
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng);
/// Обновляет максимальные значения значениями из quota.
/// Удаляет интервалы, которых нет в quota, добавляет интревалы, которых нет здесь, но есть в quota.
@ -177,7 +178,7 @@ struct Quota
Quota() : is_keyed(false), keyed_by_ip(false) {}
void loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config);
void loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng);
QuotaForIntervalsPtr get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip);
};

View File

@ -72,7 +72,7 @@ struct Settings
M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS) \
\
/** Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree */ \
M(SettingBool, optimize_move_to_prewhere, false) \
M(SettingBool, optimize_move_to_prewhere, true) \
\
/** Ожидать выполнения действий по манипуляции с партициями. 0 - не ждать, 1 - ждать выполнения только у себя, 2 - ждать всех. */ \
M(SettingUInt64, replication_alter_partitions_sync, 1) \
@ -129,6 +129,9 @@ struct Settings
* но чтобы каждый источник динамически выбирал себе доступную работу. \
*/ \
M(SettingFloat, max_streams_to_max_threads_ratio, 1) \
\
/** Позволяет выбирать метод сжатия данных при записи */\
M(SettingCompressionMethod, network_compression_method, CompressionMethod::LZ4) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;

View File

@ -4,7 +4,7 @@
#include <DB/IO/WriteHelpers.h>
#include <Poco/Timespan.h>
#include <cpuid/libcpuid.h>
#include <DB/IO/CompressedStream.h>
namespace DB
{
@ -534,5 +534,66 @@ struct SettingOverflowMode
}
};
struct SettingCompressionMethod
{
CompressionMethod value;
bool changed = false;
SettingCompressionMethod(CompressionMethod x = CompressionMethod::LZ4) : value(x) {}
operator CompressionMethod() const { return value; }
SettingCompressionMethod & operator= (CompressionMethod x) { set(x); return *this; }
static CompressionMethod getCompressionMethod(const String & s)
{
if (s == "quicklz")
return CompressionMethod::QuickLZ;
if (s == "lz4")
return CompressionMethod::LZ4;
if (s == "lz4hc")
return CompressionMethod::LZ4HC;
if (s == "zstd")
return CompressionMethod::ZSTD;
throw Exception("Unknown compression method: '" + s + "', must be one of 'quicklz', 'lz4', 'lz4hc', 'zstd' ", ErrorCodes::UNKNOWN_COMPRESSION_METHOD);
}
String toString() const
{
const char * strings[] = { "quicklz", "lz4", "lz4hc", "zstd" };
if (value < CompressionMethod::QuickLZ || value > CompressionMethod::ZSTD)
throw Exception("Unknown compression method", ErrorCodes::UNKNOWN_COMPRESSION_METHOD);
return strings[static_cast<size_t>(value)];
}
void set(CompressionMethod x)
{
value = x;
changed = true;
}
void set(const Field & x)
{
set(safeGet<const String &>(x));
}
void set(const String & x)
{
set(getCompressionMethod(x));
}
void set(ReadBuffer & buf)
{
String x;
readBinary(x, buf);
set(x);
}
void write(WriteBuffer & buf) const
{
writeBinary(toString(), buf);
}
};
}

View File

@ -9,11 +9,31 @@
namespace DB
{
/** SELECT запрос
*/
class ASTSelectQuery : public ASTQueryWithOutput
{
public:
ASTSelectQuery() = default;
ASTSelectQuery(const StringRange range_);
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "SelectQuery"; };
/// Проверить наличие функции arrayJoin. (Не большого ARRAY JOIN.)
static bool hasArrayJoin(const ASTPtr & ast);
/// Содержит ли запрос астериск?
bool hasAsterisk() const;
/// Переименовать столбцы запроса в такие же имена, как в исходном запросе.
void renameColumns(const ASTSelectQuery & source);
/// Переписывает select_expression_list, чтобы вернуть только необходимые столбцы в правильном порядке.
void rewriteSelectExpressionList(const Names & column_names);
ASTPtr clone() const override;
public:
bool distinct = false;
ASTPtr select_expression_list;
@ -32,152 +52,6 @@ public:
ASTPtr limit_offset;
ASTPtr limit_length;
ASTPtr next_union_all; /// Следующий запрос SELECT в цепочке UNION ALL, если такой есть
ASTSelectQuery() = default;
ASTSelectQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "SelectQuery"; };
/// Проверить наличие функции arrayJoin. (Не большого ARRAY JOIN.)
static bool hasArrayJoin(const ASTPtr & ast)
{
if (const ASTFunction * function = typeid_cast<const ASTFunction *>(&*ast))
if (function->kind == ASTFunction::ARRAY_JOIN)
return true;
for (const auto & child : ast->children)
if (hasArrayJoin(child))
return true;
return false;
}
/// Содержит ли запрос астериск?
bool hasAsterisk() const
{
for (const auto & ast : select_expression_list->children)
if (typeid_cast<const ASTAsterisk *>(&*ast) != nullptr)
return true;
return false;
}
/// Переименовать столбцы запроса в такие же имена, как в исходном запросе.
void renameColumns(const ASTSelectQuery & source)
{
const ASTs & from = source.select_expression_list->children;
ASTs & to = select_expression_list->children;
if (from.size() != to.size())
throw Exception("Size mismatch in UNION ALL chain",
DB::ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH);
for (size_t i = 0; i < from.size(); ++i)
{
/// Если столбец имеет алиас, то он должен совпадать с названием исходного столбца.
/// В противном случае мы ему присваиваем алиас, если требуется.
if (!to[i]->tryGetAlias().empty())
{
if (to[i]->tryGetAlias() != from[i]->getAliasOrColumnName())
throw Exception("Column alias mismatch in UNION ALL chain",
DB::ErrorCodes::UNION_ALL_COLUMN_ALIAS_MISMATCH);
}
else if (to[i]->getColumnName() != from[i]->getAliasOrColumnName())
to[i]->setAlias(from[i]->getAliasOrColumnName());
}
}
/// Переписывает select_expression_list, чтобы вернуть только необходимые столбцы в правильном порядке.
void rewriteSelectExpressionList(const Names & column_names)
{
ASTPtr result = new ASTExpressionList;
ASTs asts = select_expression_list->children;
/// Не будем выбрасывать выражения, содержащие функцию arrayJoin.
std::set<ASTPtr> unremovable_asts;
for (size_t j = 0; j < asts.size(); ++j)
{
if (hasArrayJoin(asts[j]))
{
result->children.push_back(asts[j]->clone());
unremovable_asts.insert(asts[j]);
}
}
for (const auto & column_name : column_names)
{
bool done = false;
for (size_t j = 0; j < asts.size(); ++j)
{
if (asts[j]->getAliasOrColumnName() == column_name)
{
if (!unremovable_asts.count(asts[j]))
result->children.push_back(asts[j]->clone());
done = true;
}
}
if (!done)
throw Exception("Error while rewriting expression list for select query."
" Could not find alias: " + column_name,
DB::ErrorCodes::UNKNOWN_IDENTIFIER);
}
for (auto & child : children)
{
if (child == select_expression_list)
{
child = result;
break;
}
}
select_expression_list = result;
/** NOTE: Может показаться, что мы могли испортить запрос, выбросив выражение с алиасом, который используется где-то еще.
* Такого произойти не может, потому что этот метод вызывается всегда для запроса, на котором хоть раз создавали
* ExpressionAnalyzer, что гарантирует, что в нем все алиасы уже подставлены. Не совсем очевидная логика :)
*/
}
ASTPtr clone() const override
{
ASTSelectQuery * res = new ASTSelectQuery(*this);
ASTPtr ptr{res};
res->children.clear();
#define CLONE(member) if (member) { res->member = member->clone(); res->children.push_back(res->member); }
/** NOTE Члены должны клонироваться точно в таком же порядке,
* в каком они были вставлены в children в ParserSelectQuery.
* Это важно, потому что из имён children-ов составляется идентификатор (getTreeID),
* который может быть использован для идентификаторов столбцов в случае подзапросов в операторе IN.
* При распределённой обработке запроса, в случае, если один из серверов localhost, а другой - нет,
* запрос на localhost выполняется в рамках процесса и при этом клонируется,
* а на удалённый сервер запрос отправляется в текстовом виде по TCP.
* И если порядок при клонировании не совпадает с порядком при парсинге,
* то на разных серверах получатся разные идентификаторы.
*/
CLONE(select_expression_list)
CLONE(database)
CLONE(table)
CLONE(array_join_expression_list)
CLONE(join)
CLONE(sample_size)
CLONE(prewhere_expression)
CLONE(where_expression)
CLONE(group_expression_list)
CLONE(having_expression)
CLONE(order_expression_list)
CLONE(limit_offset)
CLONE(limit_length)
CLONE(format)
CLONE(next_union_all)
#undef CLONE
return ptr;
}
};
}

View File

@ -21,7 +21,7 @@ struct ColumnsDescription
String toString() const;
static ColumnsDescription parse(const String & str, const DataTypeFactory & data_type_factory);
static ColumnsDescription parse(const String & str);
};

View File

@ -126,7 +126,7 @@ private:
const std::string & user, const std::string & password) {
return new ConnectionPool{
1, host, port, "",
user, password, storage.context.getDataTypeFactory(),
user, password,
storage.getName() + '_' + name};
};

View File

@ -1,7 +1,5 @@
#pragma once
#include <statdaemons/Increment.h>
#include <DB/Core/SortDescription.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/ExpressionActions.h>
@ -20,6 +18,9 @@
#define MERGE_TREE_MARK_SIZE (2 * sizeof(size_t))
struct SimpleIncrement;
namespace DB
{
@ -477,7 +478,7 @@ public:
}
ReadBufferFromFile file(path, std::min(static_cast<size_t>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(path).getSize()));
columns.readText(file, storage.context.getDataTypeFactory());
columns.readText(file);
}
void checkNotBroken(bool require_part_metadata)
@ -743,12 +744,12 @@ public:
* Предполагается, что кусок не пересекается с существующими.
* Если out_transaction не nullptr, присваивает туда объект, позволяющий откатить добавление куска (но не переименование).
*/
void renameTempPartAndAdd(MutableDataPartPtr & part, Increment * increment = nullptr, Transaction * out_transaction = nullptr);
void renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr);
/** То же, что renameTempPartAndAdd, но кусок может покрывать существующие куски.
* Удаляет и возвращает все куски, покрытые добавляемым (в возрастающем порядке).
*/
DataPartsVector renameTempPartAndReplace(MutableDataPartPtr & part, Increment * increment = nullptr, Transaction * out_transaction = nullptr);
DataPartsVector renameTempPartAndReplace(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr);
/** Убирает из рабочего набора куски remove и добавляет куски add. add должны уже быть в all_data_parts.
* Если clear_without_timeout, данные будут удалены при следующем clearOldParts, игнорируя old_parts_lifetime.
@ -853,13 +854,13 @@ public:
const MergeTreeSettings settings;
const ASTPtr primary_expr_ast;
Block primary_key_sample;
private:
bool require_part_metadata;
ExpressionActionsPtr primary_expr;
SortDescription sort_descr;
Block primary_key_sample;
String full_path;

View File

@ -28,8 +28,11 @@ public:
* - Проверяет правильность засечек.
* Бросает исключение, если кусок испорчен или если проверить не получилось (TODO: можно попробовать разделить эти случаи).
*/
static void checkDataPart(String path, const Settings & settings, const DataTypeFactory & data_type_factory,
MergeTreeData::DataPart::Checksums * out_checksums = nullptr);
static void checkDataPart(
String path,
const Settings & settings,
const Block & primary_key_sample, /// Проверять первичный ключ. Если не надо - передайте пустой Block.
MergeTreeData::DataPart::Checksums * out_checksums = nullptr);
};
}

View File

@ -128,13 +128,13 @@ private:
public:
Field left; /// левая граница, если есть
Field right; /// правая граница, если есть
bool left_bounded; /// ограничен ли слева
bool right_bounded; /// ограничен ли справа
bool left_included; /// включает левую границу, если есть
bool right_included; /// включает правую границу, если есть
bool left_bounded = false; /// ограничен ли слева
bool right_bounded = false; /// ограничен ли справа
bool left_included = false; /// включает левую границу, если есть
bool right_included = false; /// включает правую границу, если есть
/// Всё множество.
Range() : left(), right(), left_bounded(false), right_bounded(false), left_included(false), right_included(false) {}
Range() {}
/// Одна точка.
Range(const Field & point) : left(point), right(point), left_bounded(true), right_bounded(true), left_included(true), right_included(true) {}

View File

@ -6,6 +6,7 @@
#include <DB/Storages/MergeTree/MergeTreeDataMerger.h>
#include <DB/Storages/MergeTree/DiskSpaceMonitor.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
#include <statdaemons/Increment.h>
namespace DB
@ -107,7 +108,6 @@ private:
String database_name;
String table_name;
String full_path;
Increment increment;
Context & context;
BackgroundProcessingPool & background_pool;
@ -117,6 +117,9 @@ private:
MergeTreeDataWriter writer;
MergeTreeDataMerger merger;
/// Для нумерации блоков.
SimpleIncrement increment;
MergeTreeData::DataParts currently_merging;
Poco::FastMutex currently_merging_mutex;

View File

@ -45,7 +45,7 @@ protected:
void restore();
private:
void restoreFromFile(const String & file_path, const DataTypeFactory & data_type_factory);
void restoreFromFile(const String & file_path);
/// Вставить блок в состояние.
virtual void insertBlock(const Block & block) = 0;

View File

@ -3,6 +3,7 @@
#include <DB/TableFunctions/ITableFunction.h>
#include <DB/Storages/StorageDistributed.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/DataStreams/RemoteBlockInputStream.h>
#include <DB/Interpreters/reinterpretAsIdentifier.h>
#include <DB/Interpreters/Cluster.h>
@ -117,7 +118,7 @@ public:
if (names.empty())
throw Exception("Shard list is empty after parsing first argument", ErrorCodes::BAD_ARGUMENTS);
SharedPtr<Cluster> cluster = new Cluster(context.getSettings(), context.getDataTypeFactory(), names, username, password);
SharedPtr<Cluster> cluster = new Cluster(context.getSettings(), names, username, password);
return StorageDistributed::create(getName(), chooseColumns(*cluster, remote_database, remote_table, context),
remote_database, remote_table, cluster, context);
@ -140,6 +141,8 @@ private:
};
input->readPrefix();
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
while (true)
{
Block current = input->read();
@ -153,7 +156,7 @@ private:
String column_name = (*name)[i].get<const String &>();
String data_type_name = (*type)[i].get<const String &>();
res.emplace_back(column_name, context.getDataTypeFactory().get(data_type_name));
res.emplace_back(column_name, data_type_factory.get(data_type_name));
}
}
@ -194,7 +197,7 @@ private:
return true;
}
/* Парсит строку, генерирующую шарды и реплики. Splitter - один из двух символов | или '
/* Парсит строку, генерирующую шарды и реплики. Разделитель - один из двух символов | или ,
* в зависимости от того генерируются шарды или реплики.
* Например:
* host1,host2,... - порождает множество шардов из host1, host2, ...
@ -206,7 +209,7 @@ private:
* abc{1..9}de{f,g,h} - прямое произведение, 27 шардов.
* abc{1..9}de{0|1} - прямое произведение, 9 шардов, в каждом 2 реплики.
*/
std::vector<String> parseDescription(const String & description, size_t l, size_t r, char splitter) const
std::vector<String> parseDescription(const String & description, size_t l, size_t r, char separator) const
{
std::vector<String> res;
std::vector<String> cur;
@ -235,7 +238,7 @@ private:
if (description[m] == '{') ++cnt;
if (description[m] == '}') --cnt;
if (description[m] == '.' && description[m-1] == '.') last_dot = m;
if (description[m] == splitter) have_splitter = true;
if (description[m] == separator) have_splitter = true;
if (cnt == 0) break;
}
if (cnt != 0)
@ -279,13 +282,13 @@ private:
buffer.push_back(cur);
}
} else if (have_splitter) /// Если внутри есть текущий разделитель, то сгенерировать множество получаемых строк
buffer = parseDescription(description, i + 1, m, splitter);
buffer = parseDescription(description, i + 1, m, separator);
else /// Иначе просто скопировать, порождение произойдет при вызове с правильным разделителем
buffer.push_back(description.substr(i, m - i + 1));
/// К текущему множеству строк добавить все возможные полученные продолжения
append(cur, buffer);
i = m;
} else if (description[i] == splitter) {
} else if (description[i] == separator) {
/// Если разделитель, то добавляем в ответ найденные строки
res.insert(res.end(), cur.begin(), cur.end());
cur.clear();

View File

@ -468,7 +468,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
return res;
}
else if (name == "quantileDeterministic")
else if (name == "medianDeterministic" || name == "quantileDeterministic")
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -722,6 +722,7 @@ const AggregateFunctionFactory::FunctionNames & AggregateFunctionFactory::getFun
"quantileTimingWeighted",
"quantilesTimingWeighted",
"medianTimingWeighted",
"medianDeterministic",
"quantileDeterministic",
"quantilesDeterministic",
"sequenceMatch",

View File

@ -52,7 +52,7 @@ public:
const String & host_, UInt16 port_, const String & default_database_,
const String & user_, const String & password_, const Settings & settings_)
: concurrency(concurrency_), delay(delay_), queue(concurrency),
connections(concurrency, host_, port_, default_database_, user_, password_, data_type_factory),
connections(concurrency, host_, port_, default_database_, user_, password_),
settings(settings_), pool(concurrency)
{
std::cerr << std::fixed << std::setprecision(3);
@ -73,7 +73,6 @@ private:
typedef ConcurrentBoundedQueue<Query> Queue;
Queue queue;
DataTypeFactory data_type_factory;
ConnectionPool connections;
Settings settings;

View File

@ -336,7 +336,7 @@ private:
<< (!user.empty() ? " as user " + user : "")
<< "." << std::endl;
connection = new Connection(host, port, default_database, user, password, context.getDataTypeFactory(), "client", compression,
connection = new Connection(host, port, default_database, user, password, "client", compression,
Poco::Timespan(config().getInt("connect_timeout", DBMS_DEFAULT_CONNECT_TIMEOUT_SEC), 0),
Poco::Timespan(config().getInt("receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0),
Poco::Timespan(config().getInt("send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0));
@ -698,7 +698,7 @@ private:
current_format = insert->format;
BlockInputStreamPtr block_input = context.getFormatFactory().getInput(
current_format, buf, sample, insert_format_max_block_size, context.getDataTypeFactory());
current_format, buf, sample, insert_format_max_block_size);
BlockInputStreamPtr async_block_input = new AsynchronousBlockInputStream(block_input);

View File

@ -34,9 +34,9 @@ void Connection::connect()
if (connected)
disconnect();
LOG_TRACE(log_wrapper.get(), "Connecting to " << default_database << "@" << host << ":" << port);
LOG_TRACE(log_wrapper.get(), "Connecting. Database: " << (default_database.empty() ? "(not specified)" : default_database) << ". User: " << user);
socket.connect(Poco::Net::SocketAddress(host, port), connect_timeout);
socket.connect(resolved_address, connect_timeout);
socket.setReceiveTimeout(receive_timeout);
socket.setSendTimeout(send_timeout);
socket.setNoDelay(true);
@ -60,21 +60,21 @@ void Connection::connect()
disconnect();
/// Добавляем в сообщение адрес сервера. Также объект Exception запомнит stack trace. Жаль, что более точный тип исключения теряется.
throw NetException(e.displayText(), "(" + getServerAddress() + ")", ErrorCodes::NETWORK_ERROR);
throw NetException(e.displayText(), "(" + getDescription() + ")", ErrorCodes::NETWORK_ERROR);
}
catch (Poco::TimeoutException & e)
{
disconnect();
/// Добавляем в сообщение адрес сервера. Также объект Exception запомнит stack trace. Жаль, что более точный тип исключения теряется.
throw NetException(e.displayText(), "(" + getServerAddress() + ")", ErrorCodes::SOCKET_TIMEOUT);
throw NetException(e.displayText(), "(" + getDescription() + ")", ErrorCodes::SOCKET_TIMEOUT);
}
}
void Connection::disconnect()
{
//LOG_TRACE(log_wrapper.get(), "Disconnecting (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Disconnecting");
socket.close();
in = nullptr;
@ -85,7 +85,7 @@ void Connection::disconnect()
void Connection::sendHello()
{
//LOG_TRACE(log_wrapper.get(), "Sending hello (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Sending hello");
writeVarUInt(Protocol::Client::Hello, *out);
writeStringBinary((DBMS_NAME " ") + client_name, *out);
@ -102,7 +102,7 @@ void Connection::sendHello()
void Connection::receiveHello()
{
//LOG_TRACE(log_wrapper.get(), "Receiving hello (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Receiving hello");
/// Получить hello пакет.
UInt64 packet_type = 0;
@ -127,7 +127,7 @@ void Connection::receiveHello()
/// Закроем соединение, чтобы не было рассинхронизации.
disconnect();
throw NetException("Unexpected packet from server " + getServerAddress() + " (expected Hello or Exception, got "
throw NetException("Unexpected packet from server " + getDescription() + " (expected Hello or Exception, got "
+ String(Protocol::Server::toString(packet_type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
}
}
@ -166,33 +166,33 @@ void Connection::forceConnected()
struct PingTimeoutSetter
{
PingTimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & ping_timeout_)
PingTimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & ping_timeout_)
: socket(socket_), ping_timeout(ping_timeout_)
{
old_send_timeout = socket.getSendTimeout();
old_receive_timeout = socket.getReceiveTimeout();
if (old_send_timeout > ping_timeout)
socket.setSendTimeout(ping_timeout);
if (old_receive_timeout > ping_timeout)
socket.setReceiveTimeout(ping_timeout);
}
~PingTimeoutSetter()
{
socket.setSendTimeout(old_send_timeout);
socket.setReceiveTimeout(old_receive_timeout);
}
Poco::Net::StreamSocket & socket;
Poco::Timespan ping_timeout;
Poco::Timespan old_send_timeout;
Poco::Timespan old_receive_timeout;
};
bool Connection::ping()
{
// LOG_TRACE(log_wrapper.get(), "Ping (" << getServerAddress() << ")");
// LOG_TRACE(log_wrapper.get(), "Ping");
PingTimeoutSetter timeout_setter(socket, ping_timeout);
try
@ -219,7 +219,7 @@ bool Connection::ping()
if (pong != Protocol::Server::Pong)
{
throw Exception("Unexpected packet from server " + getServerAddress() + " (expected Pong, got "
throw Exception("Unexpected packet from server " + getDescription() + " (expected Pong, got "
+ String(Protocol::Server::toString(pong)) + ")",
ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
}
@ -236,11 +236,13 @@ bool Connection::ping()
void Connection::sendQuery(const String & query, const String & query_id_, UInt64 stage, const Settings * settings, bool with_pending_data)
{
network_compression_method = settings ? settings->network_compression_method.value : CompressionMethod::LZ4;
forceConnected();
query_id = query_id_;
//LOG_TRACE(log_wrapper.get(), "Sending query (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Sending query");
writeVarUInt(Protocol::Client::Query, *out);
@ -279,7 +281,7 @@ void Connection::sendQuery(const String & query, const String & query_id_, UInt6
void Connection::sendCancel()
{
//LOG_TRACE(log_wrapper.get(), "Sending cancel (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Sending cancel");
writeVarUInt(Protocol::Client::Cancel, *out);
out->next();
@ -288,12 +290,12 @@ void Connection::sendCancel()
void Connection::sendData(const Block & block, const String & name)
{
//LOG_TRACE(log_wrapper.get(), "Sending data (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Sending data");
if (!block_out)
{
if (compression == Protocol::Compression::Enable)
maybe_compressed_out = new CompressedWriteBuffer(*out);
maybe_compressed_out = new CompressedWriteBuffer(*out, network_compression_method);
else
maybe_compressed_out = out;
@ -403,7 +405,7 @@ bool Connection::hasReadBufferPendingData() const
Connection::Packet Connection::receivePacket()
{
//LOG_TRACE(log_wrapper.get(), "Receiving packet (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Receiving packet");
try
{
@ -446,14 +448,14 @@ Connection::Packet Connection::receivePacket()
disconnect();
throw Exception("Unknown packet "
+ toString(res.type)
+ " from server " + getServerAddress(), ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
+ " from server " + getDescription(), ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
}
catch (Exception & e)
{
/// Дописываем в текст исключения адрес сервера, если надо.
if (e.code() != ErrorCodes::UNKNOWN_PACKET_FROM_SERVER)
e.addMessage("while receiving packet from " + getServerAddress());
e.addMessage("while receiving packet from " + getDescription());
throw;
}
@ -462,7 +464,7 @@ Connection::Packet Connection::receivePacket()
Block Connection::receiveData()
{
//LOG_TRACE(log_wrapper.get(), "Receiving data (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Receiving data");
initBlockInput();
@ -492,30 +494,34 @@ void Connection::initBlockInput()
else
maybe_compressed_in = in;
block_in = new NativeBlockInputStream(*maybe_compressed_in, data_type_factory, server_revision);
block_in = new NativeBlockInputStream(*maybe_compressed_in, server_revision);
}
}
String Connection::getServerAddress() const
void Connection::setDescription()
{
return Poco::Net::SocketAddress(host, port).toString();
description = host + ":" + toString(resolved_address.port());
auto ip_address = resolved_address.host().toString();
if (host != ip_address)
description += ", " + ip_address;
}
SharedPtr<Exception> Connection::receiveException()
{
//LOG_TRACE(log_wrapper.get(), "Receiving exception (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Receiving exception");
Exception e;
readException(e, *in, "Received from " + getServerAddress());
readException(e, *in, "Received from " + getDescription());
return e.clone();
}
Progress Connection::receiveProgress()
{
//LOG_TRACE(log_wrapper.get(), "Receiving progress (" << getServerAddress() << ")");
//LOG_TRACE(log_wrapper.get(), "Receiving progress");
Progress progress;
progress.read(*in, server_revision);

View File

@ -184,7 +184,7 @@ std::string ParallelReplicas::dumpAddresses() const
const Connection * connection = e.second;
if (connection != nullptr)
{
os << (is_first ? "" : "; ") << connection->getServerAddress();
os << (is_first ? "" : "; ") << connection->getDescription();
is_first = false;
}
}

View File

@ -0,0 +1,25 @@
#include <Poco/Net/DNS.h>
#include <DB/Common/getFQDNOrHostName.h>
namespace
{
std::string getFQDNOrHostNameImpl()
{
try
{
return Poco::Net::DNS::thisHost().name();
}
catch (...)
{
return Poco::Net::DNS::hostName();
}
}
}
const std::string & getFQDNOrHostName()
{
static std::string result = getFQDNOrHostNameImpl();
return result;
}

View File

@ -1,10 +1,14 @@
#include <DB/Core/NamesAndTypes.h>
#include <DB/DataTypes/DataTypeFactory.h>
namespace DB
{
void NamesAndTypesList::readText(ReadBuffer & buf, const DataTypeFactory & data_type_factory)
void NamesAndTypesList::readText(ReadBuffer & buf)
{
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
DB::assertString("columns format version: 1\n", buf);
size_t count;
DB::readText(count, buf);
@ -45,11 +49,11 @@ String NamesAndTypesList::toString() const
return s;
}
NamesAndTypesList NamesAndTypesList::parse(const String & s, const DataTypeFactory & data_type_factory)
NamesAndTypesList NamesAndTypesList::parse(const String & s)
{
ReadBufferFromString in(s);
NamesAndTypesList res;
res.readText(in, data_type_factory);
res.readText(in);
assertEOF(in);
return res;
}

View File

@ -25,10 +25,10 @@ namespace DB
{
BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & buf,
Block & sample, size_t max_block_size, const DataTypeFactory & data_type_factory) const
Block & sample, size_t max_block_size) const
{
if (name == "Native")
return new NativeBlockInputStream(buf, data_type_factory);
return new NativeBlockInputStream(buf);
else if (name == "TabSeparated")
return new BlockInputStreamFromRowInputStream(new TabSeparatedRowInputStream(buf, sample), sample, max_block_size);
else if (name == "RowBinary")

View File

@ -4,6 +4,7 @@
#include <Poco/Ext/ThreadNumber.h>*/
#include <DB/Columns/ColumnConst.h>
#include <DB/Interpreters/Quota.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>

View File

@ -65,7 +65,7 @@ Block MergeSortingBlockInputStream::readImpl()
/// Сформируем сортированные потоки для слияния.
for (const auto & file : temporary_files)
{
temporary_inputs.emplace_back(new TemporaryFileStream(file->path(), data_type_factory));
temporary_inputs.emplace_back(new TemporaryFileStream(file->path()));
inputs_to_merge.emplace_back(temporary_inputs.back()->block_in);
}

View File

@ -5,6 +5,7 @@
#include <DB/Columns/ColumnArray.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/DataStreams/NativeBlockInputStream.h>
@ -44,6 +45,8 @@ Block NativeBlockInputStream::readImpl()
{
Block res;
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
if (istr.eof())
return res;

View File

@ -20,7 +20,7 @@ VerticalRowOutputStream::VerticalRowOutputStream(WriteBuffer & ostr_, const Bloc
typedef std::vector<size_t> Widths_t;
Widths_t name_widths(columns);
size_t max_name_width = 0;
for (size_t i = 0; i < columns; ++i)
{
data_types[i] = sample.getByPosition(i).type;
@ -41,7 +41,7 @@ void VerticalRowOutputStream::writeField(const Field & field)
writeEscapedString(names[field_number], ostr);
writeCString(": ", ostr);
writeString(pads[field_number], ostr);
data_types[field_number]->serializeTextEscaped(field, ostr);
writeChar('\n', ostr);

View File

@ -15,7 +15,6 @@
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/DataStreams/NativeBlockInputStream.h>
#include <DB/DataStreams/NativeBlockOutputStream.h>
@ -117,11 +116,9 @@ int main(int argc, char ** argv)
/// читаем данные из native файла и одновременно пишем в таблицу
if (argc == 2 && 0 == strcmp(argv[1], "write"))
{
DataTypeFactory factory;
ReadBufferFromFileDescriptor in1(STDIN_FILENO);
CompressedReadBuffer in2(in1);
NativeBlockInputStream in3(in2, factory, Revision::get());
NativeBlockInputStream in3(in2, Revision::get());
SharedPtr<IBlockOutputStream> out = table->write(0);
copyData(in3, *out);
}

View File

@ -148,11 +148,10 @@ int main(int argc, char ** argv)
sort_columns.push_back(SortColumnDescription(3, 1));
QueryProcessingStage::Enum stage;
DataTypeFactory data_type_factory;
Poco::SharedPtr<IBlockInputStream> in = table->read(column_names, 0, Context{}, Settings(), stage, argc == 2 ? atoi(argv[1]) : 1048576)[0];
in = new PartialSortingBlockInputStream(in, sort_columns);
in = new MergeSortingBlockInputStream(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, "", data_type_factory);
in = new MergeSortingBlockInputStream(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, "");
//in = new LimitBlockInputStream(in, 10);
WriteBufferFromOStream ob(std::cout);

View File

@ -29,6 +29,7 @@ void registerFunctionsStringSearch(FunctionFactory &);
void registerFunctionsURL(FunctionFactory &);
void registerFunctionsVisitParam(FunctionFactory &);
void registerFunctionsMath(FunctionFactory &);
void registerFunctionsTransform(FunctionFactory &);
FunctionFactory::FunctionFactory()
@ -55,6 +56,7 @@ FunctionFactory::FunctionFactory()
registerFunctionsURL(*this);
registerFunctionsVisitParam(*this);
registerFunctionsMath(*this);
registerFunctionsTransform(*this);
}

View File

@ -18,6 +18,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
factory.registerFunction<FunctionToStartOfQuarter>();
factory.registerFunction<FunctionToStartOfYear>();
factory.registerFunction<FunctionToStartOfMinute>();
factory.registerFunction<FunctionToStartOfFiveMinute>();
factory.registerFunction<FunctionToStartOfHour>();
factory.registerFunction<FunctionToRelativeYearNum>();
factory.registerFunction<FunctionToRelativeMonthNum>();

View File

@ -1,6 +1,5 @@
#include <math.h>
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/FunctionFactory.h>
#include <DB/Functions/FunctionsArithmetic.h>
#include <DB/Functions/FunctionsMiscellaneous.h>
@ -310,70 +309,6 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
}
/// TODO: Убрать copy-paste из FunctionsConditional.h
template <typename T>
struct DataTypeFromFieldTypeOrError
{
static DataTypePtr getDataType()
{
return new typename DataTypeFromFieldType<T>::Type;
}
};
template <>
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
{
static DataTypePtr getDataType()
{
return nullptr;
}
};
template <typename T1, typename T2>
DataTypePtr getSmallestCommonNumericTypeImpl()
{
using ResultType = typename NumberTraits::ResultOfIf<T1, T2>::Type;
auto type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
if (!type_res)
throw Exception("Types " + TypeName<T1>::get() + " and " + TypeName<T2>::get()
+ " are not upscalable to a common type without loss of precision", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_res;
}
template <typename T1>
DataTypePtr getSmallestCommonNumericTypeLeft(const IDataType & t2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt8>();
if (typeid_cast<const DataTypeUInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt16>();
if (typeid_cast<const DataTypeUInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt32>();
if (typeid_cast<const DataTypeUInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt64>();
if (typeid_cast<const DataTypeInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int8>();
if (typeid_cast<const DataTypeInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int16>();
if (typeid_cast<const DataTypeInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int32>();
if (typeid_cast<const DataTypeInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int64>();
if (typeid_cast<const DataTypeFloat32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float32>();
if (typeid_cast<const DataTypeFloat64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float64>();
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt8>(t2);
if (typeid_cast<const DataTypeUInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt16>(t2);
if (typeid_cast<const DataTypeUInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt32>(t2);
if (typeid_cast<const DataTypeUInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt64>(t2);
if (typeid_cast<const DataTypeInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int8>(t2);
if (typeid_cast<const DataTypeInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int16>(t2);
if (typeid_cast<const DataTypeInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int32>(t2);
if (typeid_cast<const DataTypeInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int64>(t2);
if (typeid_cast<const DataTypeFloat32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float32>(t2);
if (typeid_cast<const DataTypeFloat64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float64>(t2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
}
@ -404,7 +339,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
factory.registerFunction<FunctionIsInfinite>();
factory.registerFunction<FunctionIsNaN>();
factory.registerFunction<FunctionTransform>();
factory.registerFunction<FunctionVersion>();
}
}

View File

@ -20,6 +20,10 @@ void registerFunctionsString(FunctionFactory & factory)
factory.registerFunction<FunctionSubstring>();
factory.registerFunction<FunctionSubstringUTF8>();
factory.registerFunction<FunctionAppendTrailingCharIfAbsent>();
factory.registerFunction<FunctionSSELower>();
factory.registerFunction<FunctionSSEUpper>();
factory.registerFunction<FunctionSSELowerUTF8>();
factory.registerFunction<FunctionSSEUpperUTF8>();
}
}

View File

@ -0,0 +1,84 @@
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/FunctionFactory.h>
#include <DB/Functions/FunctionsTransform.h>
namespace DB
{
/// TODO: Убрать copy-paste из FunctionsConditional.h
template <typename T>
struct DataTypeFromFieldTypeOrError
{
static DataTypePtr getDataType()
{
return new typename DataTypeFromFieldType<T>::Type;
}
};
template <>
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
{
static DataTypePtr getDataType()
{
return nullptr;
}
};
template <typename T1, typename T2>
DataTypePtr getSmallestCommonNumericTypeImpl()
{
using ResultType = typename NumberTraits::ResultOfIf<T1, T2>::Type;
auto type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
if (!type_res)
throw Exception("Types " + TypeName<T1>::get() + " and " + TypeName<T2>::get()
+ " are not upscalable to a common type without loss of precision", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_res;
}
template <typename T1>
DataTypePtr getSmallestCommonNumericTypeLeft(const IDataType & t2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt8>();
if (typeid_cast<const DataTypeUInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt16>();
if (typeid_cast<const DataTypeUInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt32>();
if (typeid_cast<const DataTypeUInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt64>();
if (typeid_cast<const DataTypeInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int8>();
if (typeid_cast<const DataTypeInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int16>();
if (typeid_cast<const DataTypeInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int32>();
if (typeid_cast<const DataTypeInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int64>();
if (typeid_cast<const DataTypeFloat32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float32>();
if (typeid_cast<const DataTypeFloat64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float64>();
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt8>(t2);
if (typeid_cast<const DataTypeUInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt16>(t2);
if (typeid_cast<const DataTypeUInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt32>(t2);
if (typeid_cast<const DataTypeUInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt64>(t2);
if (typeid_cast<const DataTypeInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int8>(t2);
if (typeid_cast<const DataTypeInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int16>(t2);
if (typeid_cast<const DataTypeInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int32>(t2);
if (typeid_cast<const DataTypeInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int64>(t2);
if (typeid_cast<const DataTypeFloat32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float32>(t2);
if (typeid_cast<const DataTypeFloat64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float64>(t2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
}
namespace DB
{
void registerFunctionsTransform(FunctionFactory & factory)
{
factory.registerFunction<FunctionTransform>();
}
}

View File

@ -320,15 +320,15 @@ void WriteBufferAIO::prepare()
buffer_size += region_left_padding;
buffer_end = buffer_begin + buffer_size;
::memmove(buffer_begin + region_left_padding, buffer_begin, buffer_size - region_left_padding);
::memmove(buffer_begin + region_left_padding, buffer_begin, (buffer_size - region_left_padding) * sizeof(*buffer_begin));
ssize_t read_count = ::pread(fd, memory_page, DEFAULT_AIO_FILE_BLOCK_SIZE, region_aligned_begin);
if (read_count < 0)
throw Exception("Read error", ErrorCodes::AIO_READ_ERROR);
size_t to_copy = std::min(static_cast<size_t>(read_count), region_left_padding);
::memcpy(buffer_begin, memory_page, to_copy);
::memset(buffer_begin + to_copy, 0, region_left_padding - to_copy);
::memcpy(buffer_begin, memory_page, to_copy * sizeof(*buffer_begin));
::memset(buffer_begin + to_copy, 0, (region_left_padding - to_copy) * sizeof(*buffer_begin));
}
if (region_right_padding > 0)
@ -342,7 +342,7 @@ void WriteBufferAIO::prepare()
off_t offset = DEFAULT_AIO_FILE_BLOCK_SIZE - region_right_padding;
if (read_count > offset)
{
::memcpy(buffer_end, memory_page + offset, read_count - offset);
::memcpy(buffer_end, memory_page + offset, (read_count - offset) * sizeof(*buffer_end));
truncation_begin = buffer_end + (read_count - offset);
truncation_count = DEFAULT_AIO_FILE_BLOCK_SIZE - read_count;
}
@ -352,7 +352,7 @@ void WriteBufferAIO::prepare()
truncation_count = region_right_padding;
}
::memset(truncation_begin, 0, truncation_count);
::memset(truncation_begin, 0, truncation_count * sizeof(*truncation_begin));
}
}
}

View File

@ -13,11 +13,8 @@ Cluster::Address::Address(const String & config_prefix)
auto & config = Poco::Util::Application::instance().config();
host_name = config.getString(config_prefix + ".host");
host_port = Poco::Net::SocketAddress(
host_name,
config.getInt(config_prefix + ".port")
);
port = config.getInt(config_prefix + ".port");
resolved_address = Poco::Net::SocketAddress(host_name, port);
user = config.getString(config_prefix + ".user", "default");
password = config.getString(config_prefix + ".password", "");
}
@ -29,9 +26,17 @@ Cluster::Address::Address(const String & host_port_, const String & user_, const
/// Похоже на то, что строка host_port_ содержит порт. Если условие срабатывает - не обязательно значит, что порт есть (пример: [::]).
if (nullptr != strchr(host_port_.c_str(), ':') || !default_port)
host_port = Poco::Net::SocketAddress(host_port_);
{
resolved_address = Poco::Net::SocketAddress(host_port_);
host_name = host_port_.substr(0, host_port_.find(':'));
port = resolved_address.port();
}
else
host_port = Poco::Net::SocketAddress(host_port_, default_port);
{
resolved_address = Poco::Net::SocketAddress(host_port_, default_port);
host_name = host_port_;
port = default_port;
}
}
namespace
@ -41,13 +46,13 @@ namespace
return
escapeForFileName(address.user) +
(address.password.empty() ? "" : (':' + escapeForFileName(address.password))) + '@' +
escapeForFileName(address.host_port.host().toString()) + ':' +
std::to_string(address.host_port.port());
escapeForFileName(address.resolved_address.host().toString()) + ':' +
std::to_string(address.resolved_address.port());
}
}
Clusters::Clusters(const Settings & settings, const DataTypeFactory & data_type_factory, const String & config_name)
Clusters::Clusters(const Settings & settings, const String & config_name)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
Poco::Util::AbstractConfiguration::Keys config_keys;
@ -56,11 +61,11 @@ Clusters::Clusters(const Settings & settings, const DataTypeFactory & data_type_
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it)
impl.emplace(std::piecewise_construct,
std::forward_as_tuple(*it),
std::forward_as_tuple(settings, data_type_factory, config_name + "." + *it));
std::forward_as_tuple(settings, config_name + "." + *it));
}
Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, const String & cluster_name)
Cluster::Cluster(const Settings & settings, const String & cluster_name)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
Poco::Util::AbstractConfiguration::Keys config_keys;
@ -178,8 +183,9 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
{
replicas.emplace_back(new ConnectionPool(
settings.distributed_connections_pool_size,
replica.host_port.host().toString(), replica.host_port.port(), "", replica.user, replica.password,
data_type_factory, "server", Protocol::Compression::Enable,
replica.host_name, replica.port, replica.resolved_address,
"", replica.user, replica.password,
"server", Protocol::Compression::Enable,
saturate(settings.connect_timeout_with_failover_ms, settings.limits.max_execution_time),
saturate(settings.receive_timeout, settings.limits.max_execution_time),
saturate(settings.send_timeout, settings.limits.max_execution_time)));
@ -204,8 +210,9 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
{
pools.emplace_back(new ConnectionPool(
settings.distributed_connections_pool_size,
address.host_port.host().toString(), address.host_port.port(), "", address.user, address.password,
data_type_factory, "server", Protocol::Compression::Enable,
address.host_name, address.port, address.resolved_address,
"", address.user, address.password,
"server", Protocol::Compression::Enable,
saturate(settings.connect_timeout, settings.limits.max_execution_time),
saturate(settings.receive_timeout, settings.limits.max_execution_time),
saturate(settings.send_timeout, settings.limits.max_execution_time)));
@ -217,7 +224,7 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
}
Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, std::vector<std::vector<String>> names,
Cluster::Cluster(const Settings & settings, std::vector<std::vector<String>> names,
const String & username, const String & password)
{
for (const auto & shard : names)
@ -237,8 +244,9 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
{
replicas.emplace_back(new ConnectionPool(
settings.distributed_connections_pool_size,
replica.host_port.host().toString(), replica.host_port.port(), "", replica.user, replica.password,
data_type_factory, "server", Protocol::Compression::Enable,
replica.host_name, replica.port, replica.resolved_address,
"", replica.user, replica.password,
"server", Protocol::Compression::Enable,
saturate(settings.connect_timeout_with_failover_ms, settings.limits.max_execution_time),
saturate(settings.receive_timeout, settings.limits.max_execution_time),
saturate(settings.send_timeout, settings.limits.max_execution_time)));
@ -264,7 +272,7 @@ bool Cluster::isLocal(const Address & address)
/// - её порт совпадает с портом, который слушает сервер;
/// - её хост резолвится в набор адресов, один из которых совпадает с одним из адресов сетевых интерфейсов сервера
/// то нужно всегда ходить на этот шард без межпроцессного взаимодействия
return isLocalAddress(address.host_port);
return isLocalAddress(address.resolved_address);
}
}

View File

@ -13,7 +13,6 @@
#include <DB/DataStreams/FormatFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/TableFunctions/TableFunctionFactory.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/Storages/IStorage.h>
#include <DB/Storages/MarkCache.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
@ -72,7 +71,6 @@ struct ContextShared
Databases databases; /// Список БД и таблиц в них.
TableFunctionFactory table_function_factory; /// Табличные функции.
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
DataTypeFactory data_type_factory; /// Типы данных.
FormatFactory format_factory; /// Форматы.
mutable SharedPtr<Dictionaries> dictionaries; /// Словари Метрики. Инициализируются лениво.
mutable SharedPtr<ExternalDictionaries> external_dictionaries;
@ -155,7 +153,6 @@ Context::~Context() = default;
const TableFunctionFactory & Context::getTableFunctionFactory() const { return shared->table_function_factory; }
const AggregateFunctionFactory & Context::getAggregateFunctionFactory() const { return shared->aggregate_function_factory; }
const DataTypeFactory & Context::getDataTypeFactory() const { return shared->data_type_factory; }
const FormatFactory & Context::getFormatFactory() const { return shared->format_factory; }
InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; }
Poco::Mutex & Context::getMutex() const { return shared->mutex; }
@ -800,7 +797,7 @@ void Context::initClusters()
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (!shared->clusters)
shared->clusters = new Clusters(settings, shared->data_type_factory);
shared->clusters = new Clusters(settings);
}
Cluster & Context::getCluster(const std::string & cluster_name)

View File

@ -280,8 +280,9 @@ void ExpressionAnalyzer::addStorageAliases()
if (!storage)
return;
/// @todo: consider storing default expressions with alias set to avoid cloning
for (const auto & alias : storage->alias_columns)
aliases[alias.name] = storage->column_defaults[alias.name].expression;
(aliases[alias.name] = storage->column_defaults[alias.name].expression->clone())->setAlias(alias.name);
}
@ -768,6 +769,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
external_tables[external_table_name] = external_storage;
subqueries_for_sets[external_table_name].source = interpreter->execute();
subqueries_for_sets[external_table_name].source_sample = interpreter->getSampleBlock();
subqueries_for_sets[external_table_name].table = external_storage;
/** NOTE Если было написано IN tmp_table - существующая временная (но не внешняя) таблица,
@ -841,6 +843,7 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
{
auto interpreter = interpretSubquery(arg, context, subquery_depth);
subquery_for_set.source = new LazyBlockInputStream([interpreter]() mutable { return interpreter->execute(); });
subquery_for_set.source_sample = interpreter->getSampleBlock();
/** Зачем используется LazyBlockInputStream?
*
@ -1590,9 +1593,12 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
{
auto interpreter = interpretSubquery(ast_join.table, context, subquery_depth, required_joined_columns);
subquery_for_set.source = new LazyBlockInputStream([interpreter]() mutable { return interpreter->execute(); });
subquery_for_set.source_sample = interpreter->getSampleBlock();
}
/// TODO Это не нужно выставлять, когда JOIN нужен только на удалённых серверах.
subquery_for_set.join = join;
subquery_for_set.join->setSampleBlock(subquery_for_set.source_sample);
}
addJoinAction(step.actions, false);

View File

@ -48,7 +48,10 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
try
{
auto current = dictionary.second.first->get();
if (!dictionary.second.dict)
continue;
auto current = dictionary.second.dict->get();
const auto & lifetime = current->getLifetime();
/// do not update dictionaries with zero as lifetime
@ -75,16 +78,16 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
{
/// create new version of dictionary
auto new_version = current->clone();
dictionary.second.first->set(new_version.release());
dictionary.second.dict->set(new_version.release());
}
}
/// erase stored exception on success
stored_exceptions.erase(name);
dictionary.second.exception = std::exception_ptr{};
}
catch (...)
{
stored_exceptions.emplace(name, std::current_exception());
dictionary.second.exception = std::current_exception();
try
{
@ -162,8 +165,8 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
auto it = dictionaries.find(name);
if (it != std::end(dictionaries))
if (it->second.second != config_path)
throw std::runtime_error{"Overriding dictionary from file " + it->second.second};
if (it->second.origin != config_path)
throw std::runtime_error{"Overriding dictionary from file " + it->second.origin};
auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context);
if (!dict_ptr->isCached())
@ -184,22 +187,39 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
if (it == std::end(dictionaries))
{
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
dictionaries.emplace(name, dictionary_origin_pair_t{
dictionaries.emplace(name, dictionary_info{
std::make_shared<MultiVersion<IDictionary>>(dict_ptr.release()),
config_path
});
}
else
it->second.first->set(dict_ptr.release());
{
if (it->second.dict)
it->second.dict->set(dict_ptr.release());
else
{
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
it->second.dict = std::make_shared<MultiVersion<IDictionary>>(dict_ptr.release());
}
/// erase stored exception on success
stored_exceptions.erase(name);
/// erase stored exception on success
it->second.exception = std::exception_ptr{};
}
}
catch (...)
{
const auto exception_ptr = std::current_exception();
if (!name.empty())
stored_exceptions.emplace(name, exception_ptr);
{
const auto exception_ptr = std::current_exception();
const auto it = dictionaries.find(name);
if (it == std::end(dictionaries))
{
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
dictionaries.emplace(name, dictionary_info{nullptr, config_path, exception_ptr});
}
else
it->second.exception = exception_ptr;
}
try
{
@ -223,11 +243,28 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
/// propagate exception
if (throw_on_error)
std::rethrow_exception(exception_ptr);
throw;
}
}
}
}
}
MultiVersion<IDictionary>::Version ExternalDictionaries::getDictionary(const std::string & name) const
{
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
const auto it = dictionaries.find(name);
if (it == std::end(dictionaries))
throw Exception{
"No such dictionary: " + name,
ErrorCodes::BAD_ARGUMENTS
};
if (!it->second.dict && it->second.exception)
std::rethrow_exception(it->second.exception);
return it->second.dict->get();
}
}

View File

@ -11,6 +11,7 @@
#include <DB/IO/copyData.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/parseQuery.h>
@ -35,7 +36,7 @@ void InterpreterAlterQuery::execute()
AlterCommands alter_commands;
PartitionCommands partition_commands;
parseAlter(alter.parameters, context.getDataTypeFactory(), alter_commands, partition_commands);
parseAlter(alter.parameters, alter_commands, partition_commands);
for (const PartitionCommand & command : partition_commands)
{
@ -71,9 +72,11 @@ void InterpreterAlterQuery::execute()
}
void InterpreterAlterQuery::parseAlter(
const ASTAlterQuery::ParameterContainer & params_container, const DataTypeFactory & data_type_factory,
const ASTAlterQuery::ParameterContainer & params_container,
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands)
{
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
for (const auto & params : params_container)
{
if (params.type == ASTAlterQuery::ADD_COLUMN)

View File

@ -25,9 +25,11 @@
#include <DB/Interpreters/InterpreterSelectQuery.h>
#include <DB/Interpreters/InterpreterCreateQuery.h>
#include <DB/Interpreters/ExpressionAnalyzer.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/DataTypes/DataTypeFactory.h>
namespace DB
@ -272,6 +274,8 @@ InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(
ASTPtr default_expr_list{new ASTExpressionList};
default_expr_list->children.reserve(column_list_ast.children.size());
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
for (auto & ast : column_list_ast.children)
{
auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*ast);
@ -280,7 +284,7 @@ InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(
{
const auto & type_range = col_decl.type->range;
columns.emplace_back(col_decl.name,
context.getDataTypeFactory().get({ type_range.first, type_range.second }));
data_type_factory.get({ type_range.first, type_range.second }));
}
else
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions

View File

@ -114,8 +114,7 @@ void InterpreterInsertQuery::execute(ReadBuffer * remaining_data_istr)
BlockInputStreamPtr in{
context.getFormatFactory().getInput(
format, istr, sample, context.getSettings().max_insert_block_size,
context.getDataTypeFactory())};
format, istr, sample, context.getSettings().max_insert_block_size)};
copyData(*in, *out);
}

View File

@ -887,7 +887,7 @@ void InterpreterSelectQuery::executeOrder(BlockInputStreams & streams)
/// Сливаем сортированные блоки.
stream = new MergeSortingBlockInputStream(
stream, order_descr, settings.max_block_size, limit,
settings.limits.max_bytes_before_external_sort, context.getTemporaryPath(), context.getDataTypeFactory());
settings.limits.max_bytes_before_external_sort, context.getTemporaryPath());
}

View File

@ -323,10 +323,45 @@ void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainP
}
void Join::setSampleBlock(const Block & block)
{
Poco::ScopedWriteRWLock lock(rwlock);
if (!empty())
return;
size_t keys_size = key_names_right.size();
ConstColumnPlainPtrs key_columns(keys_size);
for (size_t i = 0; i < keys_size; ++i)
key_columns[i] = block.getByName(key_names_right[i]).column;
/// Выберем, какую структуру данных для множества использовать.
init(chooseMethod(key_columns, keys_fit_128_bits, key_sizes));
sample_block = block;
/// Удаляем из sample_block ключевые столбцы, так как они не нужны.
for (const auto & name : key_names_right)
sample_block.erase(sample_block.getPositionByName(name));
for (size_t i = 0, size = sample_block.columns(); i < size; ++i)
{
auto & column = sample_block.unsafeGetByPosition(i);
if (!column.column)
column.column = column.type->createColumn();
}
}
bool Join::insertFromBlock(const Block & block)
{
Poco::ScopedWriteRWLock lock(rwlock);
/// Какую структуру данных для множества использовать?
if (empty())
throw Exception("Logical error: Join was not initialized", ErrorCodes::LOGICAL_ERROR);
size_t keys_size = key_names_right.size();
ConstColumnPlainPtrs key_columns(keys_size);
@ -336,10 +371,6 @@ bool Join::insertFromBlock(const Block & block)
size_t rows = block.rows();
/// Какую структуру данных для множества использовать?
if (empty())
init(chooseMethod(key_columns, keys_fit_128_bits, key_sizes));
blocks.push_back(block);
Block * stored_block = &blocks.back();
@ -347,6 +378,14 @@ bool Join::insertFromBlock(const Block & block)
for (const auto & name : key_names_right)
stored_block->erase(stored_block->getPositionByName(name));
/// Редкий случай, когда соединяемые столбцы являеются константами. Чтобы не поддерживать отдельный код, материализуем их.
for (size_t i = 0, size = stored_block->columns(); i < size; ++i)
{
ColumnPtr col = stored_block->getByPosition(i).column;
if (col->isConst())
stored_block->getByPosition(i).column = dynamic_cast<IColumnConst &>(*col).convertToFullColumn();
}
if (!getFullness(kind))
{
if (strictness == ASTJoin::Any)
@ -473,9 +512,6 @@ struct Adder<KIND, ASTJoin::All, Map>
template <ASTJoin::Kind KIND, ASTJoin::Strictness STRICTNESS, typename Maps>
void Join::joinBlockImpl(Block & block, const Maps & maps) const
{
if (blocks.empty())
throw Exception("Attempt to JOIN with empty table", ErrorCodes::EMPTY_DATA_PASSED);
size_t keys_size = key_names_left.size();
ConstColumnPlainPtrs key_columns(keys_size);
@ -484,15 +520,14 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
key_columns[i] = block.getByName(key_names_left[i]).column;
/// Добавляем в блок новые столбцы.
const Block & first_mapped_block = blocks.front();
size_t num_columns_to_add = first_mapped_block.columns();
size_t num_columns_to_add = sample_block.columns();
ColumnPlainPtrs added_columns(num_columns_to_add);
size_t existing_columns = block.columns();
for (size_t i = 0; i < num_columns_to_add; ++i)
{
const ColumnWithNameAndType & src_column = first_mapped_block.getByPosition(i);
const ColumnWithNameAndType & src_column = sample_block.getByPosition(i);
ColumnWithNameAndType new_column = src_column.cloneEmpty();
block.insert(new_column);
added_columns[i] = new_column.column;
@ -630,11 +665,8 @@ void Join::joinTotals(Block & block) const
}
else
{
if (blocks.empty())
return;
/// Будем присоединять пустые totals - из одной строчки со значениями по-умолчанию.
totals_without_keys = blocks.front().cloneEmpty();
totals_without_keys = sample_block.cloneEmpty();
for (size_t i = 0; i < totals_without_keys.columns(); ++i)
{
@ -739,13 +771,12 @@ private:
}
/// Добавляем в блок новые столбцы.
const Block & first_mapped_block = parent.blocks.front();
size_t num_columns_right = first_mapped_block.columns();
size_t num_columns_right = parent.sample_block.columns();
ColumnPlainPtrs columns_right(num_columns_right);
for (size_t i = 0; i < num_columns_right; ++i)
{
const ColumnWithNameAndType & src_column = first_mapped_block.getByPosition(i);
const ColumnWithNameAndType & src_column = parent.sample_block.getByPosition(i);
ColumnWithNameAndType new_column = src_column.cloneEmpty();
block.insert(new_column);
columns_right[i] = new_column.column;

View File

@ -24,10 +24,11 @@ void QuotaValues::initFromConfig(const String & config_elem, Poco::Util::Abstrac
}
void QuotaForInterval::initFromConfig(const String & config_elem, time_t duration_, Poco::Util::AbstractConfiguration & config)
void QuotaForInterval::initFromConfig(const String & config_elem, time_t duration_, time_t offset_, Poco::Util::AbstractConfiguration & config)
{
rounded_time = 0;
duration = duration_;
offset = offset_;
max.initFromConfig(config_elem, config);
}
@ -95,7 +96,7 @@ void QuotaForInterval::updateTime(time_t current_time)
{
if (current_time >= rounded_time + static_cast<int>(duration))
{
rounded_time = current_time / duration * duration;
rounded_time = (current_time - offset) / duration * duration + offset;
used.clear();
}
}
@ -127,7 +128,7 @@ void QuotaForInterval::check(size_t max_amount, size_t used_amount, time_t curre
}
void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config)
void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng)
{
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_elem, config_keys);
@ -139,8 +140,13 @@ void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::A
String interval_config_elem = config_elem + "." + *it;
time_t duration = config.getInt(interval_config_elem + ".duration");
time_t offset = 0;
cont[duration].initFromConfig(interval_config_elem, duration, config);
bool randomize = config.getBool(interval_config_elem + ".randomize", false);
if (randomize)
offset = std::uniform_int_distribution<decltype(duration)>(0, duration - 1)(rng);
cont[duration].initFromConfig(interval_config_elem, duration, offset, config);
}
}
@ -210,7 +216,7 @@ String QuotaForIntervals::toString() const
}
void Quota::loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config)
void Quota::loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng)
{
name = name_;
@ -226,7 +232,7 @@ void Quota::loadFromConfig(const String & config_elem, const String & name_, Poc
}
QuotaForIntervals new_max(name);
new_max.initFromConfig(config_elem, config);
new_max.initFromConfig(config_elem, config, rng);
if (!(new_max == max))
{
max = new_max;
@ -269,6 +275,8 @@ QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_na
void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config)
{
std::mt19937 rng;
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys("quotas", config_keys);
@ -286,7 +294,7 @@ void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config)
{
if (!cont[*it])
cont[*it] = new Quota();
cont[*it]->loadFromConfig("quotas." + *it, *it, config);
cont[*it]->loadFromConfig("quotas." + *it, *it, config, rng);
}
}

View File

@ -3,6 +3,7 @@
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/parseQuery.h>
#include <DB/Parsers/ExpressionListParsers.h>
#include <DB/DataTypes/DataTypeFactory.h>
int main(int argc, char ** argv)
@ -25,7 +26,7 @@ int main(int argc, char ** argv)
{
NameAndTypePair col;
col.name = argv[i];
col.type = context.getDataTypeFactory().get(argv[i + 1]);
col.type = DataTypeFactory::instance().get(argv[i + 1]);
columns.push_back(col);
}

View File

@ -0,0 +1,175 @@
#include <DB/Parsers/ASTSelectQuery.h>
namespace DB
{
ASTSelectQuery::ASTSelectQuery(const StringRange range_) : ASTQueryWithOutput(range_)
{
}
bool ASTSelectQuery::hasArrayJoin(const ASTPtr & ast)
{
if (const ASTFunction * function = typeid_cast<const ASTFunction *>(&*ast))
if (function->kind == ASTFunction::ARRAY_JOIN)
return true;
for (const auto & child : ast->children)
if (hasArrayJoin(child))
return true;
return false;
}
bool ASTSelectQuery::hasAsterisk() const
{
for (const auto & ast : select_expression_list->children)
if (typeid_cast<const ASTAsterisk *>(&*ast) != nullptr)
return true;
return false;
}
void ASTSelectQuery::renameColumns(const ASTSelectQuery & source)
{
const ASTs & from = source.select_expression_list->children;
ASTs & to = select_expression_list->children;
if (from.size() != to.size())
throw Exception("Size mismatch in UNION ALL chain",
DB::ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH);
for (size_t i = 0; i < from.size(); ++i)
{
/// Если столбец имеет алиас, то он должен совпадать с названием исходного столбца.
/// В противном случае мы ему присваиваем алиас, если требуется.
if (!to[i]->tryGetAlias().empty())
{
if (to[i]->tryGetAlias() != from[i]->getAliasOrColumnName())
throw Exception("Column alias mismatch in UNION ALL chain",
DB::ErrorCodes::UNION_ALL_COLUMN_ALIAS_MISMATCH);
}
else if (to[i]->getColumnName() != from[i]->getAliasOrColumnName())
to[i]->setAlias(from[i]->getAliasOrColumnName());
}
}
void ASTSelectQuery::rewriteSelectExpressionList(const Names & column_names)
{
ASTPtr result = new ASTExpressionList;
ASTs asts = select_expression_list->children;
/// Создать отображение.
/// Элемент отображения.
struct Arrow
{
Arrow() = default;
Arrow(size_t to_position_) :
to_position(to_position_), is_selected(true)
{
}
size_t to_position = 0;
bool is_selected = false;
};
/// Отображение одного SELECT выражения в другое.
using Mapping = std::vector<Arrow>;
Mapping mapping(asts.size());
std::vector<size_t> from(column_names.size());
/// Не будем выбрасывать выражения, содержащие функцию arrayJoin.
for (size_t i = 0; i < asts.size(); ++i)
{
if (hasArrayJoin(asts[i]))
mapping[i] = Arrow(i);
}
for (size_t i = 0; i < column_names.size(); ++i)
{
bool done = false;
for (size_t j = 0; j < asts.size(); ++j)
{
if (asts[j]->getAliasOrColumnName() == column_names[i])
{
from[i] = j;
done = true;
break;
}
}
if (!done)
throw Exception("Error while rewriting expression list for select query."
" Could not find alias: " + column_names[i],
DB::ErrorCodes::UNKNOWN_IDENTIFIER);
}
auto to = from;
std::sort(from.begin(), from.end());
for (size_t i = 0; i < column_names.size(); ++i)
mapping[from[i]] = Arrow(to[i]);
/// Составить новое выражение.
for (const auto & arrow : mapping)
{
if (arrow.is_selected)
result->children.push_back(asts[arrow.to_position]->clone());
}
for (auto & child : children)
{
if (child == select_expression_list)
{
child = result;
break;
}
}
select_expression_list = result;
/** NOTE: Может показаться, что мы могли испортить запрос, выбросив выражение с алиасом, который используется где-то еще.
* Такого произойти не может, потому что этот метод вызывается всегда для запроса, на котором хоть раз создавали
* ExpressionAnalyzer, что гарантирует, что в нем все алиасы уже подставлены. Не совсем очевидная логика :)
*/
}
ASTPtr ASTSelectQuery::clone() const
{
ASTSelectQuery * res = new ASTSelectQuery(*this);
ASTPtr ptr{res};
res->children.clear();
#define CLONE(member) if (member) { res->member = member->clone(); res->children.push_back(res->member); }
/** NOTE Члены должны клонироваться точно в таком же порядке,
* в каком они были вставлены в children в ParserSelectQuery.
* Это важно, потому что из имён children-ов составляется идентификатор (getTreeID),
* который может быть использован для идентификаторов столбцов в случае подзапросов в операторе IN.
* При распределённой обработке запроса, в случае, если один из серверов localhost, а другой - нет,
* запрос на localhost выполняется в рамках процесса и при этом клонируется,
* а на удалённый сервер запрос отправляется в текстовом виде по TCP.
* И если порядок при клонировании не совпадает с порядком при парсинге,
* то на разных серверах получатся разные идентификаторы.
*/
CLONE(select_expression_list)
CLONE(database)
CLONE(table)
CLONE(array_join_expression_list)
CLONE(join)
CLONE(sample_size)
CLONE(prewhere_expression)
CLONE(where_expression)
CLONE(group_expression_list)
CLONE(having_expression)
CLONE(order_expression_list)
CLONE(limit_offset)
CLONE(limit_length)
CLONE(format)
CLONE(next_union_all)
#undef CLONE
return ptr;
}
};

View File

@ -375,6 +375,37 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr &
ws.ignore(pos, end);
/// Позволяем парсить цепочки вида NOT NOT x. Это хак.
/** Так сделано, потому что среди унарных операторов есть только минус и NOT.
* Но для минуса цепочку из унарных операторов не требуется поддерживать.
*/
if (it[0] && 0 == strncmp(it[0], "NOT", 3))
{
/// Было ли чётное количество NOT.
bool even = false;
const char ** jt;
while (true)
{
for (jt = operators; *jt; jt += 2)
{
ParserString op(jt[0], true, true);
if (op.ignore(pos, end, max_parsed_pos, expected))
break;
}
if (!*jt)
break;
even = !even;
ws.ignore(pos, end);
}
if (even)
it = jt; /// Зануляем результат парсинга первого NOT. Получается, как будто цепочки NOT нет вообще.
}
ASTPtr elem;
if (!elem_parser->parse(pos, end, elem, max_parsed_pos, expected))
return false;

View File

@ -16,7 +16,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
{
Pos begin = pos;
ASTSelectQuery * select_query = new ASTSelectQuery(StringRange(begin, pos));
ASTSelectQuery * select_query = new ASTSelectQuery;
node = select_query;
ParserWhiteSpaceOrComments ws;
@ -312,6 +312,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
ws.ignore(pos, end);
}
select_query->range = StringRange(begin, pos);
select_query->children.push_back(select_query->select_expression_list);
if (select_query->database)
select_query->children.push_back(select_query->database);

View File

@ -17,6 +17,7 @@
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Interpreters/executeQuery.h>
#include <DB/Interpreters/Quota.h>
#include <DB/Common/ExternalTable.h>

View File

@ -18,6 +18,7 @@
#include <condition_variable>
#include <DB/Common/Macros.h>
#include <DB/Common/getFQDNOrHostName.h>
#include <DB/Interpreters/loadMetadata.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/StorageSystemTables.h>
@ -485,9 +486,14 @@ int Server::main(const std::vector<std::string> & args)
{
String this_host;
if (config().has("interserver_http_host"))
{
this_host = config().getString("interserver_http_host");
}
else
this_host = Poco::Net::DNS::hostName();
{
this_host = getFQDNOrHostName();
LOG_DEBUG(log, "Configuration parameter 'interserver_http_host' doesn't exist. Will use '" + this_host + "' as replica host.");
}
String port_str = config().getString("interserver_http_port");
int port = parse<int>(port_str);

View File

@ -20,6 +20,7 @@
#include <DB/DataStreams/NativeBlockInputStream.h>
#include <DB/DataStreams/NativeBlockOutputStream.h>
#include <DB/Interpreters/executeQuery.h>
#include <DB/Interpreters/Quota.h>
#include <DB/Storages/StorageMemory.h>
@ -225,14 +226,31 @@ void TCPHandler::readData(const Settings & global_settings)
{
while (1)
{
/// Ждём пакета от клиента. При этом, каждые POLL_INTERVAL сек. проверяем, не требуется ли завершить работу.
while (!static_cast<ReadBufferFromPocoSocket &>(*in).poll(global_settings.poll_interval * 1000000) && !Daemon::instance().isCancelled())
;
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
/// Если требуется завершить работу, или клиент отсоединился.
if (Daemon::instance().isCancelled() || in->eof())
/// Ждём пакета от клиента. При этом, каждые POLL_INTERVAL сек. проверяем, не требуется ли завершить работу.
while (1)
{
if (static_cast<ReadBufferFromPocoSocket &>(*in).poll(global_settings.poll_interval * 1000000))
break;
/// Если требуется завершить работу.
if (Daemon::instance().isCancelled())
return;
/** Если ждём данных уже слишком долго.
* Если периодически poll-ить соединение, то receive_timeout у сокета сам по себе не срабатывает.
* Поэтому, добавлена дополнительная проверка.
*/
if (watch.elapsedSeconds() > global_settings.receive_timeout.totalSeconds())
throw Exception("Timeout exceeded while receiving data from client", ErrorCodes::SOCKET_TIMEOUT);
}
/// Если клиент отсоединился.
if (in->eof())
return;
/// Принимаем и обрабатываем данные. А если они закончились, то выходим.
if (!receivePacket())
break;
}
@ -606,7 +624,6 @@ void TCPHandler::initBlockInput()
state.block_in = new NativeBlockInputStream(
*state.maybe_compressed_in,
query_context.getDataTypeFactory(),
client_revision);
}
}
@ -617,7 +634,7 @@ void TCPHandler::initBlockOutput()
if (!state.block_out)
{
if (state.compression == Protocol::Compression::Enable)
state.maybe_compressed_out = new CompressedWriteBuffer(*out);
state.maybe_compressed_out = new CompressedWriteBuffer(*out, query_context.getSettings().network_compression_method);
else
state.maybe_compressed_out = out;

View File

@ -1,6 +1,7 @@
#include <DB/Parsers/ExpressionListParsers.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/Storages/ColumnsDescription.h>
#include <DB/DataTypes/DataTypeFactory.h>
namespace DB
@ -50,7 +51,7 @@ String ColumnsDescription<store>::toString() const
template <>
ColumnsDescription<true> ColumnsDescription<true>::parse(const String & str, const DataTypeFactory & data_type_factory)
ColumnsDescription<true> ColumnsDescription<true>::parse(const String & str)
{
ReadBufferFromString buf{str};
@ -60,6 +61,7 @@ ColumnsDescription<true> ColumnsDescription<true>::parse(const String & str, con
assertString(" columns:\n", buf);
ParserTernaryOperatorExpression expr_parser;
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
ColumnsDescription<true> result{};
for (size_t i = 0; i < count; ++i)

View File

@ -17,6 +17,7 @@
#include <DB/Common/localBackup.h>
#include <DB/Functions/FunctionFactory.h>
#include <Poco/DirectoryIterator.h>
#include <statdaemons/Increment.h>
#include <algorithm>
#include <iomanip>
@ -681,7 +682,7 @@ MergeTreeData::AlterDataPartTransaction::~AlterDataPartTransaction()
}
void MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, Increment * increment, Transaction * out_transaction)
void MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction)
{
auto removed = renameTempPartAndReplace(part, increment, out_transaction);
if (!removed.empty())
@ -692,7 +693,7 @@ void MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, Increment *
}
MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
MutableDataPartPtr & part, Increment * increment, Transaction * out_transaction)
MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction)
{
if (out_transaction && out_transaction->data)
throw Exception("Using the same MergeTreeData::Transaction for overlapping transactions is invalid", ErrorCodes::LOGICAL_ERROR);
@ -710,7 +711,7 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
* содержат ещё не добавленный кусок.
*/
if (increment)
part->left = part->right = increment->get(false);
part->left = part->right = increment->get();
String new_name = ActiveDataPartSet::getPartName(part->left_date, part->right_date, part->left, part->right, part->level);
@ -770,6 +771,7 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
if (obsolete)
{
LOG_WARNING(log, "Obsolete part " + part->name + " added");
part->remove_time = time(0);
}
else
{
@ -813,7 +815,9 @@ void MergeTreeData::attachPart(const DataPartPtr & part)
if (!all_data_parts.insert(part).second)
throw Exception("Part " + part->name + " is already attached", ErrorCodes::DUPLICATE_DATA_PART);
data_parts.insert(part);
addPartContributionToColumnSizes(part);
}
void MergeTreeData::renameAndDetachPart(const DataPartPtr & part, const String & prefix, bool restore_covered, bool move_to_detached)
@ -1037,7 +1041,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const St
MergeTreePartChecker::Settings settings;
settings.setIndexGranularity(index_granularity);
settings.setRequireColumnFiles(true);
MergeTreePartChecker::checkDataPart(full_path + relative_path, settings, context.getDataTypeFactory(), &part->checksums);
MergeTreePartChecker::checkDataPart(full_path + relative_path, settings, primary_key_sample, &part->checksums);
{
WriteBufferFromFile out(full_path + relative_path + "/checksums.txt.tmp", 4096);

View File

@ -165,7 +165,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
UInt64 sampling_column_value_lower_limit;
UInt64 sampling_column_value_upper_limit;
UInt64 upper_limit = static_cast<UInt64>(relative_sample_size * sampling_column_max);
UInt64 upper_limit = static_cast<UInt64>(static_cast<long double>(relative_sample_size) * sampling_column_max);
if (settings.parallel_replicas_count > 1)
{

View File

@ -249,8 +249,11 @@ static size_t checkColumn(const String & path, const String & name, DataTypePtr
}
}
void MergeTreePartChecker::checkDataPart(String path, const Settings & settings, const DataTypeFactory & data_type_factory,
MergeTreeData::DataPart::Checksums * out_checksums)
void MergeTreePartChecker::checkDataPart(
String path,
const Settings & settings,
const Block & primary_key_sample,
MergeTreeData::DataPart::Checksums * out_checksums)
{
if (!path.empty() && path.back() != '/')
path += "/";
@ -262,7 +265,7 @@ void MergeTreePartChecker::checkDataPart(String path, const Settings & settings,
{
ReadBufferFromFile buf(path + "columns.txt");
columns.readText(buf, data_type_factory);
columns.readText(buf);
assertEOF(buf);
}
@ -275,12 +278,30 @@ void MergeTreePartChecker::checkDataPart(String path, const Settings & settings,
/// Реальные чексуммы по содержимому данных. Их несоответствие checksums_txt будет говорить о битых данных.
MergeTreeData::DataPart::Checksums checksums_data;
size_t primary_idx_size;
size_t marks_in_primary_key = 0;
{
ReadBufferFromFile file_buf(path + "primary.idx");
HashingReadBuffer hashing_buf(file_buf);
primary_idx_size = hashing_buf.tryIgnore(std::numeric_limits<size_t>::max());
if (primary_key_sample)
{
Field tmp_field;
size_t key_size = primary_key_sample.columns();
while (!hashing_buf.eof())
{
++marks_in_primary_key;
for (size_t j = 0; j < key_size; ++j)
primary_key_sample.unsafeGetByPosition(j).type->deserializeBinary(tmp_field, hashing_buf);
}
}
else
{
hashing_buf.tryIgnore(std::numeric_limits<size_t>::max());
}
size_t primary_idx_size = hashing_buf.count();
checksums_data.files["primary.idx"] = MergeTreeData::DataPart::Checksums::Checksum(primary_idx_size, hashing_buf.getHash());
}
@ -345,9 +366,17 @@ void MergeTreePartChecker::checkDataPart(String path, const Settings & settings,
if (rows == Stream::UNKNOWN)
throw Exception("No columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
if (primary_idx_size % ((rows - 1) / settings.index_granularity + 1))
throw Exception("primary.idx size (" + toString(primary_idx_size) + ") not divisible by number of marks ("
+ toString(rows) + "/" + toString(settings.index_granularity) + " rounded up)", ErrorCodes::CORRUPTED_DATA);
if (primary_key_sample)
{
const size_t expected_marks = (rows - 1) / settings.index_granularity + 1;
if (expected_marks != marks_in_primary_key)
throw Exception("Size of primary key doesn't match expected number of marks."
" Number of rows in columns: " + toString(rows)
+ ", index_granularity: " + toString(settings.index_granularity)
+ ", expected number of marks: " + toString(expected_marks)
+ ", size of primary key: " + toString(marks_in_primary_key),
ErrorCodes::CORRUPTED_DATA);
}
if (settings.require_checksums || !checksums_txt.files.empty())
checksums_txt.checkEqual(checksums_data, true);

View File

@ -33,6 +33,7 @@ StorageJoin::StorageJoin(
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE};
join = new Join(key_names, key_names, Limits(), kind, strictness);
join->setSampleBlock(getSampleBlock());
restore();
}

View File

@ -130,6 +130,12 @@ BlockInputStreams StorageMerge::read(
std::multiset<String> values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_table");
/** На всякий случай отключаем оптимизацию "перенос в PREWHERE",
* так как нет уверенности, что она работает, когда одна из таблиц MergeTree, а другая - нет.
*/
Settings modified_settings = settings;
modified_settings.optimize_move_to_prewhere = false;
for (size_t i = 0, size = selected_tables.size(); i < size; ++i)
{
StoragePtr table = selected_tables[i];
@ -150,7 +156,7 @@ BlockInputStreams StorageMerge::read(
real_column_names,
modified_query_ast,
context,
settings,
modified_settings,
tmp_processed_stage,
max_block_size,
size > threads ? 1 : (threads / size));

View File

@ -29,20 +29,20 @@ StorageMergeTree::StorageMergeTree(
const MergeTreeSettings & settings_)
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
path(path_), database_name(database_name_), table_name(table_name_), full_path(path + escapeForFileName(table_name) + '/'),
increment(full_path + "increment.txt"), context(context_), background_pool(context_.getBackgroundPool()),
context(context_), background_pool(context_.getBackgroundPool()),
data(full_path, columns_,
materialized_columns_, alias_columns_, column_defaults_,
context_, primary_expr_ast_, date_column_name_,
sampling_expression_, index_granularity_,mode_, sign_column_, columns_to_sum_,
settings_, database_name_ + "." + table_name, false),
reader(data), writer(data), merger(data),
increment(0),
log(&Logger::get(database_name_ + "." + table_name + " (StorageMergeTree)")),
shutdown_called(false)
{
increment.fixIfBroken(data.getMaxDataPartIndex());
data.loadDataParts(false);
data.clearOldParts();
increment.set(data.getMaxDataPartIndex());
}
StoragePtr StorageMergeTree::create(
@ -130,8 +130,6 @@ void StorageMergeTree::rename(const String & new_path_to_db, const String & new_
table_name = new_table_name;
full_path = new_full_path;
increment.setPath(full_path + "increment.txt");
/// TODO: Можно обновить названия логгеров у this, data, reader, writer, merger.
}
@ -341,16 +339,10 @@ void StorageMergeTree::attachPartition(const Field & field, bool unreplicated, b
LOG_DEBUG(log, "Checking data");
MergeTreeData::MutableDataPartPtr part = data.loadPartAndFixMetadata(source_path);
UInt64 index = increment.get();
String new_part_name = ActiveDataPartSet::getPartName(part->left_date, part->right_date, index, index, 0);
part->renameTo(new_part_name);
part->name = new_part_name;
ActiveDataPartSet::parsePartName(part->name, *part);
LOG_INFO(log, "Attaching part " << source_part_name << " from " << source_path);
data.renameTempPartAndAdd(part, &increment);
LOG_INFO(log, "Attaching part " << source_part_name << " from " << source_path << " as " << new_part_name);
data.attachPart(part);
LOG_INFO(log, "Finished attaching part " << new_part_name);
LOG_INFO(log, "Finished attaching part");
}
/// На месте удаленных кусков могут появиться новые, с другими данными.

View File

@ -268,8 +268,7 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo
assertEOF(buf);
zkutil::Stat stat;
auto columns_desc = ColumnsDescription<true>::parse(
zookeeper->get(zookeeper_path + "/columns", &stat), context.getDataTypeFactory());
auto columns_desc = ColumnsDescription<true>::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
auto & columns = columns_desc.columns;
auto & materialized_columns = columns_desc.materialized;
@ -1459,7 +1458,7 @@ void StorageReplicatedMergeTree::alterThread()
zkutil::Stat stat;
const String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
auto columns_desc = ColumnsDescription<true>::parse(columns_str, context.getDataTypeFactory());
auto columns_desc = ColumnsDescription<true>::parse(columns_str);
auto & columns = columns_desc.columns;
auto & materialized_columns = columns_desc.materialized;
@ -1809,7 +1808,7 @@ void StorageReplicatedMergeTree::partCheckThread()
zk_checksums.checkEqual(part->checksums, true);
auto zk_columns = NamesAndTypesList::parse(
zookeeper->get(replica_path + "/parts/" + part_name + "/columns"), context.getDataTypeFactory());
zookeeper->get(replica_path + "/parts/" + part_name + "/columns"));
if (part->columns != zk_columns)
throw Exception("Columns of local part " + part_name + " are different from ZooKeeper");
@ -1818,7 +1817,7 @@ void StorageReplicatedMergeTree::partCheckThread()
settings.setRequireChecksums(true);
settings.setRequireColumnFiles(true);
MergeTreePartChecker::checkDataPart(
data.getFullPath() + part_name, settings, context.getDataTypeFactory());
data.getFullPath() + part_name, settings, data.primary_key_sample);
LOG_INFO(log, "Part " << part_name << " looks good.");
}
@ -2512,54 +2511,138 @@ void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String &
{
auto zookeeper = getZooKeeper();
UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
String log_entry_str = entry.toString();
String entry_str = entry.toString();
String log_node_name;
LOG_DEBUG(log, "Waiting for " << replica << " to pull " << entry.znode_name << " to queue");
/** В эту функцию могут передать entry двух видов:
* 1. (более часто) Из директории log - общего лога, откуда реплики копируют записи в свою queue.
* 2. Из директории queue одной из реплик.
*
* Проблема в том, что номера (sequential нод) элементов очереди в log и в queue не совпадают.
* (И в queue не совпадают номера у одного и того же элемента лога для разных реплик.)
*
* Поэтому следует рассматривать эти случаи по-отдельности.
*/
/// Дождемся, пока запись попадет в очередь реплики.
while (true)
/** Первое - нужно дождаться, пока реплика возьмёт к себе в queue элемент очереди из log,
* если она ещё этого не сделала (см. функцию pullLogsToQueue).
*
* Для этого проверяем её узел log_pointer - максимальный номер взятого элемента из log плюс единица.
*/
if (0 == entry.znode_name.compare(0, strlen("log-"), "log-"))
{
zkutil::EventPtr event = new Poco::Event;
/** В этом случае просто берём номер из имени ноды log-xxxxxxxxxx.
*/
String pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event);
if (!pointer.empty() && parse<UInt64>(pointer) > log_index)
break;
UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
log_node_name = entry.znode_name;
event->wait();
LOG_DEBUG(log, "Waiting for " << replica << " to pull " << log_node_name << " to queue");
/// Дождемся, пока запись попадет в очередь реплики.
while (true)
{
zkutil::EventPtr event = new Poco::Event;
String log_pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event);
if (!log_pointer.empty() && parse<UInt64>(log_pointer) > log_index)
break;
event->wait();
}
}
else if (0 == entry.znode_name.compare(0, strlen("queue-"), "queue-"))
{
/** В этом случае номер log-ноды неизвестен. Нужно просмотреть все от log_pointer до конца,
* ища ноду с таким же содержимым. И если мы её не найдём - значит реплика уже взяла эту запись в свою queue.
*/
LOG_DEBUG(log, "Looking for " << entry.znode_name << " in " << replica << " queue");
String log_pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer");
Strings log_entries = zookeeper->getChildren(zookeeper_path + "/log");
UInt64 log_index = 0;
bool found = false;
for (const String & log_entry_name : log_entries)
{
log_index = parse<UInt64>(log_entry_name.substr(log_entry_name.size() - 10));
if (!log_pointer.empty() && log_index < parse<UInt64>(log_pointer))
continue;
String log_entry_str;
bool exists = zookeeper->tryGet(zookeeper_path + "/log/" + log_entry_name, log_entry_str);
if (exists && entry_str == log_entry_str)
{
found = true;
log_node_name = log_entry_name;
break;
}
}
if (found)
{
LOG_DEBUG(log, "Waiting for " << replica << " to pull " << log_node_name << " to queue");
/// Дождемся, пока запись попадет в очередь реплики.
while (true)
{
zkutil::EventPtr event = new Poco::Event;
String log_pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event);
if (!log_pointer.empty() && parse<UInt64>(log_pointer) > log_index)
break;
event->wait();
}
}
}
else
throw Exception("Logical error: unexpected name of log node: " + entry.znode_name, ErrorCodes::LOGICAL_ERROR);
if (!log_node_name.empty())
LOG_DEBUG(log, "Looking for node corresponding to " << log_node_name << " in " << replica << " queue");
else
LOG_DEBUG(log, "Looking for corresponding node in " << replica << " queue");
/** Второе - найдем соответствующую запись в очереди указанной реплики (replica).
* Её номер может не совпадать ни с log-узлом, ни с queue-узлом у текущей реплики (у нас).
* Поэтому, ищем путём сравнения содержимого.
*/
/// Найдем запись в очереди реплики.
Strings queue_entries = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/queue");
String entry_to_wait_for;
String queue_entry_to_wait_for;
for (const String & entry_name : queue_entries)
{
String queue_entry_str;
bool exists = zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + entry_name, queue_entry_str);
if (exists && queue_entry_str == log_entry_str)
if (exists && queue_entry_str == entry_str)
{
entry_to_wait_for = entry_name;
queue_entry_to_wait_for = entry_name;
break;
}
}
/// Пока искали запись, ее уже выполнили и удалили.
if (entry_to_wait_for.empty())
if (queue_entry_to_wait_for.empty())
{
LOG_DEBUG(log, "No corresponding node found. Assuming it has been already processed.");
return;
}
LOG_DEBUG(log, "Waiting for " << entry_to_wait_for << " to disappear from " << replica << " queue");
LOG_DEBUG(log, "Waiting for " << queue_entry_to_wait_for << " to disappear from " << replica << " queue");
/// Третье - дождемся, пока запись исчезнет из очереди реплики.
/// Дождемся, пока запись исчезнет из очереди реплики.
while (true)
{
zkutil::EventPtr event = new Poco::Event;
String unused;
/// get вместо exists, чтобы не утек watch, если ноды уже нет.
if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + entry_to_wait_for, unused, nullptr, event))
if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for, unused, nullptr, event))
break;
event->wait();

View File

@ -87,8 +87,6 @@ void StorageSetOrJoinBase::restore()
constexpr auto file_suffix = ".bin";
constexpr auto file_suffix_size = strlen(file_suffix);
DataTypeFactory data_type_factory;
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator dir_it(path); dir_end != dir_it; ++dir_it)
{
@ -104,17 +102,17 @@ void StorageSetOrJoinBase::restore()
if (file_num > increment)
increment = file_num;
restoreFromFile(dir_it->path(), data_type_factory);
restoreFromFile(dir_it->path());
}
}
}
void StorageSetOrJoinBase::restoreFromFile(const String & file_path, const DataTypeFactory & data_type_factory)
void StorageSetOrJoinBase::restoreFromFile(const String & file_path)
{
ReadBufferFromFile backup_buf(file_path);
CompressedReadBuffer compressed_backup_buf(backup_buf);
NativeBlockInputStream backup_stream(compressed_backup_buf, data_type_factory);
NativeBlockInputStream backup_stream(compressed_backup_buf);
backup_stream.readPrefix();
while (Block block = backup_stream.read())

View File

@ -62,8 +62,8 @@ BlockInputStreams StorageSystemClusters::read(
replica_num_column->insert(static_cast<UInt64>(address.replica_num));
host_name_column->insert(address.host_name);
host_address_column->insert(address.host_port.host().toString());
port_column->insert(static_cast<UInt64>(address.host_port.port()));
host_address_column->insert(address.resolved_address.host().toString());
port_column->insert(static_cast<UInt64>(address.port));
user_column->insert(address.user);
};

View File

@ -21,8 +21,8 @@ StorageSystemDictionaries::StorageSystemDictionaries(const std::string & name)
: name{name},
columns{
{ "name", new DataTypeString },
{ "type", new DataTypeString },
{ "origin", new DataTypeString },
{ "type", new DataTypeString },
{ "attribute.names", new DataTypeArray{new DataTypeString} },
{ "attribute.types", new DataTypeArray{new DataTypeString} },
{ "has_hierarchy", new DataTypeUInt8 },
@ -56,8 +56,8 @@ BlockInputStreams StorageSystemDictionaries::read(
processed_stage = QueryProcessingStage::FetchColumns;
ColumnWithNameAndType col_name{new ColumnString, new DataTypeString, "name"};
ColumnWithNameAndType col_type{new ColumnString, new DataTypeString, "type"};
ColumnWithNameAndType col_origin{new ColumnString, new DataTypeString, "origin"};
ColumnWithNameAndType col_type{new ColumnString, new DataTypeString, "type"};
ColumnWithNameAndType col_attribute_names{
new ColumnArray{new ColumnString},
new DataTypeArray{new DataTypeString},
@ -83,34 +83,51 @@ BlockInputStreams StorageSystemDictionaries::read(
for (const auto & dict_info : external_dictionaries.dictionaries)
{
const auto & name = dict_info.first;
const auto dict_ptr = dict_info.second.first->get();
col_name.column->insert(dict_info.first);
col_origin.column->insert(dict_info.second.origin);
col_name.column->insert(name);
col_type.column->insert(dict_ptr->getTypeName());
col_origin.column->insert(dict_info.second.second);
if (dict_info.second.dict)
{
const auto dict_ptr = dict_info.second.dict->get();
const auto & dict_struct = dict_ptr->getStructure();
col_attribute_names.column->insert(ext::map<Array>(dict_struct.attributes, [] (auto & attr) -> decltype(auto) {
return attr.name;
}));
col_attribute_types.column->insert(ext::map<Array>(dict_struct.attributes, [] (auto & attr) -> decltype(auto) {
return attr.type->getName();
}));
col_has_hierarchy.column->insert(UInt64{dict_ptr->hasHierarchy()});
col_bytes_allocated.column->insert(dict_ptr->getBytesAllocated());
col_query_count.column->insert(dict_ptr->getQueryCount());
col_hit_rate.column->insert(dict_ptr->getHitRate());
col_element_count.column->insert(dict_ptr->getElementCount());
col_load_factor.column->insert(dict_ptr->getLoadFactor());
col_creation_time.column->insert(std::chrono::system_clock::to_time_t(dict_ptr->getCreationTime()));
col_type.column->insert(dict_ptr->getTypeName());
const auto exception_it = external_dictionaries.stored_exceptions.find(name);
if (exception_it != std::end(external_dictionaries.stored_exceptions))
const auto & dict_struct = dict_ptr->getStructure();
col_attribute_names.column->insert(ext::map<Array>(dict_struct.attributes, [] (auto & attr) -> decltype(auto) {
return attr.name;
}));
col_attribute_types.column->insert(ext::map<Array>(dict_struct.attributes, [] (auto & attr) -> decltype(auto) {
return attr.type->getName();
}));
col_has_hierarchy.column->insert(UInt64{dict_ptr->hasHierarchy()});
col_bytes_allocated.column->insert(dict_ptr->getBytesAllocated());
col_query_count.column->insert(dict_ptr->getQueryCount());
col_hit_rate.column->insert(dict_ptr->getHitRate());
col_element_count.column->insert(dict_ptr->getElementCount());
col_load_factor.column->insert(dict_ptr->getLoadFactor());
col_creation_time.column->insert(std::chrono::system_clock::to_time_t(dict_ptr->getCreationTime()));
col_source.column->insert(dict_ptr->getSource()->toString());
}
else
{
col_type.column->insertDefault();
col_attribute_names.column->insertDefault();
col_attribute_types.column->insertDefault();
col_has_hierarchy.column->insertDefault();
col_bytes_allocated.column->insertDefault();
col_query_count.column->insertDefault();
col_hit_rate.column->insertDefault();
col_element_count.column->insertDefault();
col_load_factor.column->insertDefault();
col_creation_time.column->insertDefault();
col_source.column->insertDefault();
}
if (dict_info.second.exception)
{
try
{
std::rethrow_exception(exception_it->second);
std::rethrow_exception(dict_info.second.exception);
}
catch (const Exception & e)
{
@ -130,15 +147,13 @@ BlockInputStreams StorageSystemDictionaries::read(
}
}
else
col_last_exception.column->insert(std::string{});
col_source.column->insert(dict_ptr->getSource()->toString());
col_last_exception.column->insertDefault();
}
Block block{
col_name,
col_type,
col_origin,
col_type,
col_attribute_names,
col_attribute_types,
col_has_hierarchy,

View File

@ -4,6 +4,8 @@
int main(int argc, char ** argv)
{
using namespace DB;
Poco::AutoPtr<Poco::ConsoleChannel> channel = new Poco::ConsoleChannel(std::cerr);
Logger::root().setChannel(channel);
Logger::root().setLevel("trace");
@ -16,18 +18,18 @@ int main(int argc, char ** argv)
try
{
DB::MergeTreePartChecker::Settings settings;
MergeTreePartChecker::Settings settings;
if (argc == 4)
settings.setIndexGranularity(DB::parse<size_t>(argv[3]));
settings.setIndexGranularity(parse<size_t>(argv[3]));
settings.setRequireChecksums(argv[2][0] == '1');
settings.setRequireColumnFiles(argv[2][0] == '1');
settings.setVerbose(true);
DB::MergeTreePartChecker::checkDataPart(argv[1], settings, DB::DataTypeFactory());
MergeTreePartChecker::checkDataPart(argv[1], settings, Block());
}
catch (...)
{
DB::tryLogCurrentException(__PRETTY_FUNCTION__);
tryLogCurrentException(__PRETTY_FUNCTION__);
throw;
}

View File

@ -1,10 +1,10 @@
0 0 0 Hello
1 0.5 0 Hello
1 0.5 0
2 1 3 Hello
3 1.5 0 Hello
3 1.5 0
4 2 6 Hello
5 2.5 0 Hello
5 2.5 0
6 3 9 Hello
7 3.5 0 Hello
8 4 0 Hello
9 4.5 0 Hello
7 3.5 0
8 4 0
9 4.5 0

View File

@ -0,0 +1,4 @@
0 -1
0 -1
-1 0
-1 0

View File

@ -0,0 +1,2 @@
SELECT * FROM (SELECT dummy, -1 as x UNION ALL SELECT dummy, arrayJoin([-1]) as x);
SELECT * FROM (SELECT -1 as x, dummy UNION ALL SELECT arrayJoin([-1]) as x, dummy);

View File

@ -0,0 +1,857 @@
0 0 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
3 3 2
2 2 2
-3 -2 -3
-2 -2 -3
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 0 0
0 0 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 0 0
0 0 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
13 13 13
10 20 10
10 20 10
10 20 10
10 20 10
13 20 10
13 20 10
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
0 100 0
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-10 -20 -10
-10 -20 -10
-10 -20 -10
-10 -20 -10
-13 -20 -10
-13 -20 -10
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 0 0
0 0 0
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-10 -20 -10
-10 -20 -10
-10 -20 -10
-10 -20 -10
-13 -20 -10
-13 -20 -10
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-10 -20 -10
-10 -20 -10
-10 -20 -10
-10 -20 -10
-13 -20 -10
-13 -20 -10
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-10 -20 -10
-10 -20 -10
-10 -20 -10
-10 -20 -10
-13 -20 -10
-13 -20 -10
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-10 -20 -10
-10 -20 -10
-10 -20 -10
-10 -20 -10
-13 -20 -10
-13 -20 -10
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-13 -13 -13
-10 -20 -10
-10 -20 -10
-10 -20 -10
-10 -20 -10
-13 -20 -10
-13 -20 -10
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
0 -100 0
2.72 2.72 2.71
2.72 2.72 2.71
2.72 2.72 2.71
2.72 2.72 2.71
2.72 2.72 2.71
2.72 2.72 2.71
2.72 2.72 2.71
2.72 2.72 2.71
2.72 2.72 2.71
2.72 2.72 2.71
2.7 2.8 2.7
2.7 2.8 2.7
2.7 2.8 2.7
2.7 2.8 2.7
2.7 2.8 2.7
2.7 2.8 2.7
2.7 2.8 2.7
2.7 2.8 2.7
2.7 2.8 2.7
2.7 2.8 2.7
3 3 2
3 3 2
3 3 2
3 3 2
3 3 2
3 3 2
3 3 2
3 3 2
3 3 2
0 10 0
0 10 0
0 10 0
0 10 0
2.7 10 0
2.7 10 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
-2.72 -2.71 -2.72
-2.72 -2.71 -2.72
-2.72 -2.71 -2.72
-2.72 -2.71 -2.72
-2.72 -2.71 -2.72
-2.72 -2.71 -2.72
-2.72 -2.71 -2.72
-2.72 -2.71 -2.72
-2.72 -2.71 -2.72
-2.72 -2.71 -2.72
-2.7 -2.7 -2.8
-2.7 -2.7 -2.8
-2.7 -2.7 -2.8
-2.7 -2.7 -2.8
-2.7 -2.7 -2.8
-2.7 -2.7 -2.8
-2.7 -2.7 -2.8
-2.7 -2.7 -2.8
-2.7 -2.7 -2.8
-2.7 -2.7 -2.8
-3 -2 -3
-3 -2 -3
-3 -2 -3
-3 -2 -3
-3 -2 -3
-3 -2 -3
-3 -2 -3
-3 -2 -3
-3 -2 -3
0 -10 0
0 -10 0
0 -10 0
0 -10 0
-2.7 -10 0
-2.7 -10 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
13112220 13112230 13112220
13112200 13112300 13112200
13112000 13113000 13112000
13110000 13120000 13110000
13100000 13200000 13100000
13000000 14000000 13000000
10000000 20000000 10000000
0 100000000 0
0 1000000000 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
2.7 2.8 2.7
2.72 2.72 2.71
2.718 2.719 2.718
2.7183 2.7183 2.7182
2.71828 2.71829 2.71828
2.718282 2.718282 2.718281
2.7182818 2.7182819 2.7182818
2.71828183 2.71828183 2.71828182
2.718281828 2.718281829 2.718281828
2.7182818285 2.7182818285 2.7182818284
2.71828182846 2.71828182846 2.71828182845
2.718281828459 2.71828182846 2.718281828459
2.718281828459 2.7182818284591 2.718281828459
2.71828182845904 2.71828182845905 2.71828182845904
2.718281828459045 2.718281828459045 2.718281828459045
2.718281828459045 2.718281828459045 2.718281828459045
2.718281828459045 2.718281828459045 2.718281828459045
2.718281828459045 2.718281828459045 2.718281828459045
2.718281828459045 2.718281828459045 2.718281828459045
2.718281828459045 2.718281828459045 2.718281828459045
3.718
3.718
3.218
3.718
3.218
2.918
3.718
3.218
2.918
2.818
3.718
3.218
2.918
2.818
2.777
3.718
3.218
2.918
2.818
2.777
2.757
3.718
3.218
2.918
2.818
2.777
2.757
2.745
3.718
3.218
2.918
2.818
2.777
2.757
2.745
2.738
3.718
3.218
2.918
2.818
2.777
2.757
2.745
2.738
2.734
3.718
3.218
2.918
2.818
2.777
2.757
2.745
2.738
2.734
2.73
0
0
0

View File

@ -0,0 +1,954 @@
/* Без дополнительного параметра */
SELECT round(0), ceil(0), floor(0);
SELECT round(toUInt8(13)), ceil(toUInt8(13)), floor(toUInt8(13));
SELECT round(toUInt16(13)), ceil(toUInt16(13)), floor(toUInt16(13));
SELECT round(toUInt32(13)), ceil(toUInt32(13)), floor(toUInt32(13));
SELECT round(toUInt64(13)), ceil(toUInt64(13)), floor(toUInt64(13));
SELECT round(toInt8(13)), ceil(toInt8(13)), floor(toInt8(13));
SELECT round(toInt16(13)), ceil(toInt16(13)), floor(toInt16(13));
SELECT round(toInt32(13)), ceil(toInt32(13)), floor(toInt32(13));
SELECT round(toInt64(13)), ceil(toInt64(13)), floor(toInt64(13));
SELECT round(toFloat32(13)), ceil(toFloat32(13)), floor(toFloat32(13));
SELECT round(toFloat64(13)), ceil(toFloat64(13)), floor(toFloat64(13));
SELECT round(toInt8(-13)), ceil(toInt8(-13)), floor(toInt8(-13));
SELECT round(toInt16(-13)), ceil(toInt16(-13)), floor(toInt16(-13));
SELECT round(toInt32(-13)), ceil(toInt32(-13)), floor(toInt32(-13));
SELECT round(toInt64(-13)), ceil(toInt64(-13)), floor(toInt64(-13));
SELECT round(toFloat32(-13)), ceil(toFloat32(-13)), floor(toFloat32(-13));
SELECT round(toFloat64(-13)), ceil(toFloat64(-13)), floor(toFloat64(-13));
SELECT round(2.7), ceil(2.7), floor(2.7);
SELECT round(2.1), ceil(2,1), floor(2.1);
SELECT round(-2.7), ceil(-2.7), floor(-2.7);
SELECT round(-2.1), ceil(-2,1), floor(-2.1);
/* UInt8 */
SELECT round(toUInt8(13), toUInt8(2)), ceil(toUInt8(13), toUInt8(2)), floor(toUInt8(13), toUInt8(2));
SELECT round(toUInt8(13), toUInt16(2)), ceil(toUInt8(13), toUInt16(2)), floor(toUInt8(13), toUInt16(2));
SELECT round(toUInt8(13), toUInt32(2)), ceil(toUInt8(13), toUInt32(2)), floor(toUInt8(13), toUInt32(2));
SELECT round(toUInt8(13), toUInt64(2)), ceil(toUInt8(13), toUInt64(2)), floor(toUInt8(13), toUInt64(2));
SELECT round(toUInt8(13), toInt8(2)), ceil(toUInt8(13), toInt8(2)), floor(toUInt8(13), toInt8(2));
SELECT round(toUInt8(13), toInt16(2)), ceil(toUInt8(13), toInt16(2)), floor(toUInt8(13), toInt16(2));
SELECT round(toUInt8(13), toInt32(2)), ceil(toUInt8(13), toInt32(2)), floor(toUInt8(13), toInt32(2));
SELECT round(toUInt8(13), toInt64(2)), ceil(toUInt8(13), toInt64(2)), floor(toUInt8(13), toInt64(2));
SELECT round(toUInt8(13), toFloat32(2.1)), ceil(toUInt8(13), toFloat32(2.1)), floor(toUInt8(13), toFloat32(2.1));
SELECT round(toUInt8(13), toFloat64(2.1)), ceil(toUInt8(13), toFloat64(2.1)), floor(toUInt8(13), toFloat64(2.1));
SELECT round(toUInt8(13), toUInt8(1)), ceil(toUInt8(13), toUInt8(1)), floor(toUInt8(13), toUInt8(1));
SELECT round(toUInt8(13), toUInt16(1)), ceil(toUInt8(13), toUInt16(1)), floor(toUInt8(13), toUInt16(1));
SELECT round(toUInt8(13), toUInt32(1)), ceil(toUInt8(13), toUInt32(1)), floor(toUInt8(13), toUInt32(1));
SELECT round(toUInt8(13), toUInt64(1)), ceil(toUInt8(13), toUInt64(1)), floor(toUInt8(13), toUInt64(1));
SELECT round(toUInt8(13), toInt8(1)), ceil(toUInt8(13), toInt8(1)), floor(toUInt8(13), toInt8(1));
SELECT round(toUInt8(13), toInt16(1)), ceil(toUInt8(13), toInt16(1)), floor(toUInt8(13), toInt16(1));
SELECT round(toUInt8(13), toInt32(1)), ceil(toUInt8(13), toInt32(1)), floor(toUInt8(13), toInt32(1));
SELECT round(toUInt8(13), toInt64(1)), ceil(toUInt8(13), toInt64(1)), floor(toUInt8(13), toInt64(1));
SELECT round(toUInt8(13), toFloat32(1.1)), ceil(toUInt8(13), toFloat32(1.1)), floor(toUInt8(13), toFloat32(1.1));
SELECT round(toUInt8(13), toFloat64(1.1)), ceil(toUInt8(13), toFloat64(1.1)), floor(toUInt8(13), toFloat64(1.1));
SELECT round(toUInt8(13), toUInt16(0)), ceil(toUInt8(13), toUInt16(0)), floor(toUInt8(13), toUInt16(0));
SELECT round(toUInt8(13), toUInt32(0)), ceil(toUInt8(13), toUInt32(0)), floor(toUInt8(13), toUInt32(0));
SELECT round(toUInt8(13), toUInt64(0)), ceil(toUInt8(13), toUInt64(0)), floor(toUInt8(13), toUInt64(0));
SELECT round(toUInt8(13), toInt8(0)), ceil(toUInt8(13), toInt8(0)), floor(toUInt8(13), toInt8(0));
SELECT round(toUInt8(13), toInt16(0)), ceil(toUInt8(13), toInt16(0)), floor(toUInt8(13), toInt16(0));
SELECT round(toUInt8(13), toInt32(0)), ceil(toUInt8(13), toInt32(0)), floor(toUInt8(13), toInt32(0));
SELECT round(toUInt8(13), toInt64(0)), ceil(toUInt8(13), toInt64(0)), floor(toUInt8(13), toInt64(0));
SELECT round(toUInt8(13), toFloat32(0.1)), ceil(toUInt8(13), toFloat32(0.1)), floor(toUInt8(13), toFloat32(0.1));
SELECT round(toUInt8(13), toFloat64(0.1)), ceil(toUInt8(13), toFloat64(0.1)), floor(toUInt8(13), toFloat64(0.1));
SELECT round(toUInt8(13), toInt8(-1)), ceil(toUInt8(13), toInt8(-1)), floor(toUInt8(13), toInt8(-1));
SELECT round(toUInt8(13), toInt16(-1)), ceil(toUInt8(13), toInt16(-1)), floor(toUInt8(13), toInt16(-1));
SELECT round(toUInt8(13), toInt32(-1)), ceil(toUInt8(13), toInt32(-1)), floor(toUInt8(13), toInt32(-1));
SELECT round(toUInt8(13), toInt64(-1)), ceil(toUInt8(13), toInt64(-1)), floor(toUInt8(13), toInt64(-1));
SELECT round(toUInt8(13), toFloat32(1.1)), ceil(toUInt8(13), toFloat32(-1.1)), floor(toUInt8(13), toFloat32(-1.1));
SELECT round(toUInt8(13), toFloat64(1.1)), ceil(toUInt8(13), toFloat64(-1.1)), floor(toUInt8(13), toFloat64(-1.1));
SELECT round(toUInt8(13), toInt8(-2)), ceil(toUInt8(13), toInt8(-2)), floor(toUInt8(13), toInt8(-2));
SELECT round(toUInt8(13), toInt16(-2)), ceil(toUInt8(13), toInt16(-2)), floor(toUInt8(13), toInt16(-2));
SELECT round(toUInt8(13), toInt32(-2)), ceil(toUInt8(13), toInt32(-2)), floor(toUInt8(13), toInt32(-2));
SELECT round(toUInt8(13), toInt64(-2)), ceil(toUInt8(13), toInt64(-2)), floor(toUInt8(13), toInt64(-2));
SELECT round(toUInt8(13), toFloat32(-2.1)), ceil(toUInt8(13), toFloat32(-2.1)), floor(toUInt8(13), toFloat32(-2.1));
SELECT round(toUInt8(13), toFloat64(-2.1)), ceil(toUInt8(13), toFloat64(-2.1)), floor(toUInt8(13), toFloat64(-2.1));
/* UInt16 */
SELECT round(toUInt16(13), toUInt8(2)), ceil(toUInt16(13), toUInt8(2)), floor(toUInt16(13), toUInt8(2));
SELECT round(toUInt16(13), toUInt16(2)), ceil(toUInt16(13), toUInt16(2)), floor(toUInt16(13), toUInt16(2));
SELECT round(toUInt16(13), toUInt32(2)), ceil(toUInt16(13), toUInt32(2)), floor(toUInt16(13), toUInt32(2));
SELECT round(toUInt16(13), toUInt64(2)), ceil(toUInt16(13), toUInt64(2)), floor(toUInt16(13), toUInt64(2));
SELECT round(toUInt16(13), toInt8(2)), ceil(toUInt16(13), toInt8(2)), floor(toUInt16(13), toInt8(2));
SELECT round(toUInt16(13), toInt16(2)), ceil(toUInt16(13), toInt16(2)), floor(toUInt16(13), toInt16(2));
SELECT round(toUInt16(13), toInt32(2)), ceil(toUInt16(13), toInt32(2)), floor(toUInt16(13), toInt32(2));
SELECT round(toUInt16(13), toInt64(2)), ceil(toUInt16(13), toInt64(2)), floor(toUInt16(13), toInt64(2));
SELECT round(toUInt16(13), toFloat32(2.1)), ceil(toUInt16(13), toFloat32(2.1)), floor(toUInt16(13), toFloat32(2.1));
SELECT round(toUInt16(13), toFloat64(2.1)), ceil(toUInt16(13), toFloat64(2.1)), floor(toUInt16(13), toFloat64(2.1));
SELECT round(toUInt16(13), toUInt8(1)), ceil(toUInt16(13), toUInt8(1)), floor(toUInt16(13), toUInt8(1));
SELECT round(toUInt16(13), toUInt16(1)), ceil(toUInt16(13), toUInt16(1)), floor(toUInt16(13), toUInt16(1));
SELECT round(toUInt16(13), toUInt32(1)), ceil(toUInt16(13), toUInt32(1)), floor(toUInt16(13), toUInt32(1));
SELECT round(toUInt16(13), toUInt64(1)), ceil(toUInt16(13), toUInt64(1)), floor(toUInt16(13), toUInt64(1));
SELECT round(toUInt16(13), toInt8(1)), ceil(toUInt16(13), toInt8(1)), floor(toUInt16(13), toInt8(1));
SELECT round(toUInt16(13), toInt16(1)), ceil(toUInt16(13), toInt16(1)), floor(toUInt16(13), toInt16(1));
SELECT round(toUInt16(13), toInt32(1)), ceil(toUInt16(13), toInt32(1)), floor(toUInt16(13), toInt32(1));
SELECT round(toUInt16(13), toInt64(1)), ceil(toUInt16(13), toInt64(1)), floor(toUInt16(13), toInt64(1));
SELECT round(toUInt16(13), toFloat32(1.1)), ceil(toUInt16(13), toFloat32(1.1)), floor(toUInt16(13), toFloat32(1.1));
SELECT round(toUInt16(13), toFloat64(1.1)), ceil(toUInt16(13), toFloat64(1.1)), floor(toUInt16(13), toFloat64(1.1));
SELECT round(toUInt16(13), toUInt16(0)), ceil(toUInt16(13), toUInt16(0)), floor(toUInt16(13), toUInt16(0));
SELECT round(toUInt16(13), toUInt32(0)), ceil(toUInt16(13), toUInt32(0)), floor(toUInt16(13), toUInt32(0));
SELECT round(toUInt16(13), toUInt64(0)), ceil(toUInt16(13), toUInt64(0)), floor(toUInt16(13), toUInt64(0));
SELECT round(toUInt16(13), toInt8(0)), ceil(toUInt16(13), toInt8(0)), floor(toUInt16(13), toInt8(0));
SELECT round(toUInt16(13), toInt16(0)), ceil(toUInt16(13), toInt16(0)), floor(toUInt16(13), toInt16(0));
SELECT round(toUInt16(13), toInt32(0)), ceil(toUInt16(13), toInt32(0)), floor(toUInt16(13), toInt32(0));
SELECT round(toUInt16(13), toInt64(0)), ceil(toUInt16(13), toInt64(0)), floor(toUInt16(13), toInt64(0));
SELECT round(toUInt16(13), toFloat32(0.1)), ceil(toUInt16(13), toFloat32(0.1)), floor(toUInt16(13), toFloat32(0.1));
SELECT round(toUInt16(13), toFloat64(0.1)), ceil(toUInt16(13), toFloat64(0.1)), floor(toUInt16(13), toFloat64(0.1));
SELECT round(toUInt16(13), toInt8(-1)), ceil(toUInt16(13), toInt8(-1)), floor(toUInt16(13), toInt8(-1));
SELECT round(toUInt16(13), toInt16(-1)), ceil(toUInt16(13), toInt16(-1)), floor(toUInt16(13), toInt16(-1));
SELECT round(toUInt16(13), toInt32(-1)), ceil(toUInt16(13), toInt32(-1)), floor(toUInt16(13), toInt32(-1));
SELECT round(toUInt16(13), toInt64(-1)), ceil(toUInt16(13), toInt64(-1)), floor(toUInt16(13), toInt64(-1));
SELECT round(toUInt16(13), toFloat32(1.1)), ceil(toUInt16(13), toFloat32(-1.1)), floor(toUInt16(13), toFloat32(-1.1));
SELECT round(toUInt16(13), toFloat64(1.1)), ceil(toUInt16(13), toFloat64(-1.1)), floor(toUInt16(13), toFloat64(-1.1));
SELECT round(toUInt16(13), toInt8(-2)), ceil(toUInt16(13), toInt8(-2)), floor(toUInt16(13), toInt8(-2));
SELECT round(toUInt16(13), toInt16(-2)), ceil(toUInt16(13), toInt16(-2)), floor(toUInt16(13), toInt16(-2));
SELECT round(toUInt16(13), toInt32(-2)), ceil(toUInt16(13), toInt32(-2)), floor(toUInt16(13), toInt32(-2));
SELECT round(toUInt16(13), toInt64(-2)), ceil(toUInt16(13), toInt64(-2)), floor(toUInt16(13), toInt64(-2));
SELECT round(toUInt16(13), toFloat32(-2.1)), ceil(toUInt16(13), toFloat32(-2.1)), floor(toUInt16(13), toFloat32(-2.1));
SELECT round(toUInt16(13), toFloat64(-2.1)), ceil(toUInt16(13), toFloat64(-2.1)), floor(toUInt16(13), toFloat64(-2.1));
/* UInt32 */
SELECT round(toUInt32(13), toUInt8(2)), ceil(toUInt32(13), toUInt8(2)), floor(toUInt32(13), toUInt8(2));
SELECT round(toUInt32(13), toUInt16(2)), ceil(toUInt32(13), toUInt16(2)), floor(toUInt32(13), toUInt16(2));
SELECT round(toUInt32(13), toUInt32(2)), ceil(toUInt32(13), toUInt32(2)), floor(toUInt32(13), toUInt32(2));
SELECT round(toUInt32(13), toUInt64(2)), ceil(toUInt32(13), toUInt64(2)), floor(toUInt32(13), toUInt64(2));
SELECT round(toUInt32(13), toInt8(2)), ceil(toUInt32(13), toInt8(2)), floor(toUInt32(13), toInt8(2));
SELECT round(toUInt32(13), toInt16(2)), ceil(toUInt32(13), toInt16(2)), floor(toUInt32(13), toInt16(2));
SELECT round(toUInt32(13), toInt32(2)), ceil(toUInt32(13), toInt32(2)), floor(toUInt32(13), toInt32(2));
SELECT round(toUInt32(13), toInt64(2)), ceil(toUInt32(13), toInt64(2)), floor(toUInt32(13), toInt64(2));
SELECT round(toUInt32(13), toFloat32(2.1)), ceil(toUInt32(13), toFloat32(2.1)), floor(toUInt32(13), toFloat32(2.1));
SELECT round(toUInt32(13), toFloat64(2.1)), ceil(toUInt32(13), toFloat64(2.1)), floor(toUInt32(13), toFloat64(2.1));
SELECT round(toUInt32(13), toUInt8(1)), ceil(toUInt32(13), toUInt8(1)), floor(toUInt32(13), toUInt8(1));
SELECT round(toUInt32(13), toUInt16(1)), ceil(toUInt32(13), toUInt16(1)), floor(toUInt32(13), toUInt16(1));
SELECT round(toUInt32(13), toUInt32(1)), ceil(toUInt32(13), toUInt32(1)), floor(toUInt32(13), toUInt32(1));
SELECT round(toUInt32(13), toUInt64(1)), ceil(toUInt32(13), toUInt64(1)), floor(toUInt32(13), toUInt64(1));
SELECT round(toUInt32(13), toInt8(1)), ceil(toUInt32(13), toInt8(1)), floor(toUInt32(13), toInt8(1));
SELECT round(toUInt32(13), toInt16(1)), ceil(toUInt32(13), toInt16(1)), floor(toUInt32(13), toInt16(1));
SELECT round(toUInt32(13), toInt32(1)), ceil(toUInt32(13), toInt32(1)), floor(toUInt32(13), toInt32(1));
SELECT round(toUInt32(13), toInt64(1)), ceil(toUInt32(13), toInt64(1)), floor(toUInt32(13), toInt64(1));
SELECT round(toUInt32(13), toFloat32(1.1)), ceil(toUInt32(13), toFloat32(1.1)), floor(toUInt32(13), toFloat32(1.1));
SELECT round(toUInt32(13), toFloat64(1.1)), ceil(toUInt32(13), toFloat64(1.1)), floor(toUInt32(13), toFloat64(1.1));
SELECT round(toUInt32(13), toUInt16(0)), ceil(toUInt32(13), toUInt16(0)), floor(toUInt32(13), toUInt16(0));
SELECT round(toUInt32(13), toUInt32(0)), ceil(toUInt32(13), toUInt32(0)), floor(toUInt32(13), toUInt32(0));
SELECT round(toUInt32(13), toUInt64(0)), ceil(toUInt32(13), toUInt64(0)), floor(toUInt32(13), toUInt64(0));
SELECT round(toUInt32(13), toInt8(0)), ceil(toUInt32(13), toInt8(0)), floor(toUInt32(13), toInt8(0));
SELECT round(toUInt32(13), toInt16(0)), ceil(toUInt32(13), toInt16(0)), floor(toUInt32(13), toInt16(0));
SELECT round(toUInt32(13), toInt32(0)), ceil(toUInt32(13), toInt32(0)), floor(toUInt32(13), toInt32(0));
SELECT round(toUInt32(13), toInt64(0)), ceil(toUInt32(13), toInt64(0)), floor(toUInt32(13), toInt64(0));
SELECT round(toUInt32(13), toFloat32(0.1)), ceil(toUInt32(13), toFloat32(0.1)), floor(toUInt32(13), toFloat32(0.1));
SELECT round(toUInt32(13), toFloat64(0.1)), ceil(toUInt32(13), toFloat64(0.1)), floor(toUInt32(13), toFloat64(0.1));
SELECT round(toUInt32(13), toInt8(-1)), ceil(toUInt32(13), toInt8(-1)), floor(toUInt32(13), toInt8(-1));
SELECT round(toUInt32(13), toInt16(-1)), ceil(toUInt32(13), toInt16(-1)), floor(toUInt32(13), toInt16(-1));
SELECT round(toUInt32(13), toInt32(-1)), ceil(toUInt32(13), toInt32(-1)), floor(toUInt32(13), toInt32(-1));
SELECT round(toUInt32(13), toInt64(-1)), ceil(toUInt32(13), toInt64(-1)), floor(toUInt32(13), toInt64(-1));
SELECT round(toUInt32(13), toFloat32(1.1)), ceil(toUInt32(13), toFloat32(-1.1)), floor(toUInt32(13), toFloat32(-1.1));
SELECT round(toUInt32(13), toFloat64(1.1)), ceil(toUInt32(13), toFloat64(-1.1)), floor(toUInt32(13), toFloat64(-1.1));
SELECT round(toUInt32(13), toInt8(-2)), ceil(toUInt32(13), toInt8(-2)), floor(toUInt32(13), toInt8(-2));
SELECT round(toUInt32(13), toInt16(-2)), ceil(toUInt32(13), toInt16(-2)), floor(toUInt32(13), toInt16(-2));
SELECT round(toUInt32(13), toInt32(-2)), ceil(toUInt32(13), toInt32(-2)), floor(toUInt32(13), toInt32(-2));
SELECT round(toUInt32(13), toInt64(-2)), ceil(toUInt32(13), toInt64(-2)), floor(toUInt32(13), toInt64(-2));
SELECT round(toUInt32(13), toFloat32(-2.1)), ceil(toUInt32(13), toFloat32(-2.1)), floor(toUInt32(13), toFloat32(-2.1));
SELECT round(toUInt32(13), toFloat64(-2.1)), ceil(toUInt32(13), toFloat64(-2.1)), floor(toUInt32(13), toFloat64(-2.1));
/* UInt64 */
SELECT round(toUInt64(13), toUInt8(2)), ceil(toUInt64(13), toUInt8(2)), floor(toUInt64(13), toUInt8(2));
SELECT round(toUInt64(13), toUInt16(2)), ceil(toUInt64(13), toUInt16(2)), floor(toUInt64(13), toUInt16(2));
SELECT round(toUInt64(13), toUInt32(2)), ceil(toUInt64(13), toUInt32(2)), floor(toUInt64(13), toUInt32(2));
SELECT round(toUInt64(13), toUInt64(2)), ceil(toUInt64(13), toUInt64(2)), floor(toUInt64(13), toUInt64(2));
SELECT round(toUInt64(13), toInt8(2)), ceil(toUInt64(13), toInt8(2)), floor(toUInt64(13), toInt8(2));
SELECT round(toUInt64(13), toInt16(2)), ceil(toUInt64(13), toInt16(2)), floor(toUInt64(13), toInt16(2));
SELECT round(toUInt64(13), toInt32(2)), ceil(toUInt64(13), toInt32(2)), floor(toUInt64(13), toInt32(2));
SELECT round(toUInt64(13), toInt64(2)), ceil(toUInt64(13), toInt64(2)), floor(toUInt64(13), toInt64(2));
SELECT round(toUInt64(13), toFloat32(2.1)), ceil(toUInt64(13), toFloat32(2.1)), floor(toUInt64(13), toFloat32(2.1));
SELECT round(toUInt64(13), toFloat64(2.1)), ceil(toUInt64(13), toFloat64(2.1)), floor(toUInt64(13), toFloat64(2.1));
SELECT round(toUInt64(13), toUInt8(1)), ceil(toUInt64(13), toUInt8(1)), floor(toUInt64(13), toUInt8(1));
SELECT round(toUInt64(13), toUInt16(1)), ceil(toUInt64(13), toUInt16(1)), floor(toUInt64(13), toUInt16(1));
SELECT round(toUInt64(13), toUInt32(1)), ceil(toUInt64(13), toUInt32(1)), floor(toUInt64(13), toUInt32(1));
SELECT round(toUInt64(13), toUInt64(1)), ceil(toUInt64(13), toUInt64(1)), floor(toUInt64(13), toUInt64(1));
SELECT round(toUInt64(13), toInt8(1)), ceil(toUInt64(13), toInt8(1)), floor(toUInt64(13), toInt8(1));
SELECT round(toUInt64(13), toInt16(1)), ceil(toUInt64(13), toInt16(1)), floor(toUInt64(13), toInt16(1));
SELECT round(toUInt64(13), toInt32(1)), ceil(toUInt64(13), toInt32(1)), floor(toUInt64(13), toInt32(1));
SELECT round(toUInt64(13), toInt64(1)), ceil(toUInt64(13), toInt64(1)), floor(toUInt64(13), toInt64(1));
SELECT round(toUInt64(13), toFloat32(1.1)), ceil(toUInt64(13), toFloat32(1.1)), floor(toUInt64(13), toFloat32(1.1));
SELECT round(toUInt64(13), toFloat64(1.1)), ceil(toUInt64(13), toFloat64(1.1)), floor(toUInt64(13), toFloat64(1.1));
SELECT round(toUInt64(13), toUInt16(0)), ceil(toUInt64(13), toUInt16(0)), floor(toUInt64(13), toUInt16(0));
SELECT round(toUInt64(13), toUInt32(0)), ceil(toUInt64(13), toUInt32(0)), floor(toUInt64(13), toUInt32(0));
SELECT round(toUInt64(13), toUInt64(0)), ceil(toUInt64(13), toUInt64(0)), floor(toUInt64(13), toUInt64(0));
SELECT round(toUInt64(13), toInt8(0)), ceil(toUInt64(13), toInt8(0)), floor(toUInt64(13), toInt8(0));
SELECT round(toUInt64(13), toInt16(0)), ceil(toUInt64(13), toInt16(0)), floor(toUInt64(13), toInt16(0));
SELECT round(toUInt64(13), toInt32(0)), ceil(toUInt64(13), toInt32(0)), floor(toUInt64(13), toInt32(0));
SELECT round(toUInt64(13), toInt64(0)), ceil(toUInt64(13), toInt64(0)), floor(toUInt64(13), toInt64(0));
SELECT round(toUInt64(13), toFloat32(0.1)), ceil(toUInt64(13), toFloat32(0.1)), floor(toUInt64(13), toFloat32(0.1));
SELECT round(toUInt64(13), toFloat64(0.1)), ceil(toUInt64(13), toFloat64(0.1)), floor(toUInt64(13), toFloat64(0.1));
SELECT round(toUInt64(13), toInt8(-1)), ceil(toUInt64(13), toInt8(-1)), floor(toUInt64(13), toInt8(-1));
SELECT round(toUInt64(13), toInt16(-1)), ceil(toUInt64(13), toInt16(-1)), floor(toUInt64(13), toInt16(-1));
SELECT round(toUInt64(13), toInt32(-1)), ceil(toUInt64(13), toInt32(-1)), floor(toUInt64(13), toInt32(-1));
SELECT round(toUInt64(13), toInt64(-1)), ceil(toUInt64(13), toInt64(-1)), floor(toUInt64(13), toInt64(-1));
SELECT round(toUInt64(13), toFloat32(1.1)), ceil(toUInt64(13), toFloat32(-1.1)), floor(toUInt64(13), toFloat32(-1.1));
SELECT round(toUInt64(13), toFloat64(1.1)), ceil(toUInt64(13), toFloat64(-1.1)), floor(toUInt64(13), toFloat64(-1.1));
SELECT round(toUInt64(13), toInt8(-2)), ceil(toUInt64(13), toInt8(-2)), floor(toUInt64(13), toInt8(-2));
SELECT round(toUInt64(13), toInt16(-2)), ceil(toUInt64(13), toInt16(-2)), floor(toUInt64(13), toInt16(-2));
SELECT round(toUInt64(13), toInt32(-2)), ceil(toUInt64(13), toInt32(-2)), floor(toUInt64(13), toInt32(-2));
SELECT round(toUInt64(13), toInt64(-2)), ceil(toUInt64(13), toInt64(-2)), floor(toUInt64(13), toInt64(-2));
SELECT round(toUInt64(13), toFloat32(-2.1)), ceil(toUInt64(13), toFloat32(-2.1)), floor(toUInt64(13), toFloat32(-2.1));
SELECT round(toUInt64(13), toFloat64(-2.1)), ceil(toUInt64(13), toFloat64(-2.1)), floor(toUInt64(13), toFloat64(-2.1));
/* Int8 */
SELECT round(toInt8(13), toUInt8(2)), ceil(toInt8(13), toUInt8(2)), floor(toInt8(13), toUInt8(2));
SELECT round(toInt8(13), toUInt16(2)), ceil(toInt8(13), toUInt16(2)), floor(toInt8(13), toUInt16(2));
SELECT round(toInt8(13), toUInt32(2)), ceil(toInt8(13), toUInt32(2)), floor(toInt8(13), toUInt32(2));
SELECT round(toInt8(13), toUInt64(2)), ceil(toInt8(13), toUInt64(2)), floor(toInt8(13), toUInt64(2));
SELECT round(toInt8(13), toInt8(2)), ceil(toInt8(13), toInt8(2)), floor(toInt8(13), toInt8(2));
SELECT round(toInt8(13), toInt16(2)), ceil(toInt8(13), toInt16(2)), floor(toInt8(13), toInt16(2));
SELECT round(toInt8(13), toInt32(2)), ceil(toInt8(13), toInt32(2)), floor(toInt8(13), toInt32(2));
SELECT round(toInt8(13), toInt64(2)), ceil(toInt8(13), toInt64(2)), floor(toInt8(13), toInt64(2));
SELECT round(toInt8(13), toFloat32(2.1)), ceil(toInt8(13), toFloat32(2.1)), floor(toInt8(13), toFloat32(2.1));
SELECT round(toInt8(13), toFloat64(2.1)), ceil(toInt8(13), toFloat64(2.1)), floor(toInt8(13), toFloat64(2.1));
SELECT round(toInt8(13), toUInt8(1)), ceil(toInt8(13), toUInt8(1)), floor(toInt8(13), toUInt8(1));
SELECT round(toInt8(13), toUInt16(1)), ceil(toInt8(13), toUInt16(1)), floor(toInt8(13), toUInt16(1));
SELECT round(toInt8(13), toUInt32(1)), ceil(toInt8(13), toUInt32(1)), floor(toInt8(13), toUInt32(1));
SELECT round(toInt8(13), toUInt64(1)), ceil(toInt8(13), toUInt64(1)), floor(toInt8(13), toUInt64(1));
SELECT round(toInt8(13), toInt8(1)), ceil(toInt8(13), toInt8(1)), floor(toInt8(13), toInt8(1));
SELECT round(toInt8(13), toInt16(1)), ceil(toInt8(13), toInt16(1)), floor(toInt8(13), toInt16(1));
SELECT round(toInt8(13), toInt32(1)), ceil(toInt8(13), toInt32(1)), floor(toInt8(13), toInt32(1));
SELECT round(toInt8(13), toInt64(1)), ceil(toInt8(13), toInt64(1)), floor(toInt8(13), toInt64(1));
SELECT round(toInt8(13), toFloat32(1.1)), ceil(toInt8(13), toFloat32(1.1)), floor(toInt8(13), toFloat32(1.1));
SELECT round(toInt8(13), toFloat64(1.1)), ceil(toInt8(13), toFloat64(1.1)), floor(toInt8(13), toFloat64(1.1));
SELECT round(toInt8(13), toUInt16(0)), ceil(toInt8(13), toUInt16(0)), floor(toInt8(13), toUInt16(0));
SELECT round(toInt8(13), toUInt32(0)), ceil(toInt8(13), toUInt32(0)), floor(toInt8(13), toUInt32(0));
SELECT round(toInt8(13), toUInt64(0)), ceil(toInt8(13), toUInt64(0)), floor(toInt8(13), toUInt64(0));
SELECT round(toInt8(13), toInt8(0)), ceil(toInt8(13), toInt8(0)), floor(toInt8(13), toInt8(0));
SELECT round(toInt8(13), toInt16(0)), ceil(toInt8(13), toInt16(0)), floor(toInt8(13), toInt16(0));
SELECT round(toInt8(13), toInt32(0)), ceil(toInt8(13), toInt32(0)), floor(toInt8(13), toInt32(0));
SELECT round(toInt8(13), toInt64(0)), ceil(toInt8(13), toInt64(0)), floor(toInt8(13), toInt64(0));
SELECT round(toInt8(13), toFloat32(0.1)), ceil(toInt8(13), toFloat32(0.1)), floor(toInt8(13), toFloat32(0.1));
SELECT round(toInt8(13), toFloat64(0.1)), ceil(toInt8(13), toFloat64(0.1)), floor(toInt8(13), toFloat64(0.1));
SELECT round(toInt8(13), toInt8(-1)), ceil(toInt8(13), toInt8(-1)), floor(toInt8(13), toInt8(-1));
SELECT round(toInt8(13), toInt16(-1)), ceil(toInt8(13), toInt16(-1)), floor(toInt8(13), toInt16(-1));
SELECT round(toInt8(13), toInt32(-1)), ceil(toInt8(13), toInt32(-1)), floor(toInt8(13), toInt32(-1));
SELECT round(toInt8(13), toInt64(-1)), ceil(toInt8(13), toInt64(-1)), floor(toInt8(13), toInt64(-1));
SELECT round(toInt8(13), toFloat32(1.1)), ceil(toInt8(13), toFloat32(-1.1)), floor(toInt8(13), toFloat32(-1.1));
SELECT round(toInt8(13), toFloat64(1.1)), ceil(toInt8(13), toFloat64(-1.1)), floor(toInt8(13), toFloat64(-1.1));
SELECT round(toInt8(13), toInt8(-2)), ceil(toInt8(13), toInt8(-2)), floor(toInt8(13), toInt8(-2));
SELECT round(toInt8(13), toInt16(-2)), ceil(toInt8(13), toInt16(-2)), floor(toInt8(13), toInt16(-2));
SELECT round(toInt8(13), toInt32(-2)), ceil(toInt8(13), toInt32(-2)), floor(toInt8(13), toInt32(-2));
SELECT round(toInt8(13), toInt64(-2)), ceil(toInt8(13), toInt64(-2)), floor(toInt8(13), toInt64(-2));
SELECT round(toInt8(13), toFloat32(-2.1)), ceil(toInt8(13), toFloat32(-2.1)), floor(toInt8(13), toFloat32(-2.1));
SELECT round(toInt8(13), toFloat64(-2.1)), ceil(toInt8(13), toFloat64(-2.1)), floor(toInt8(13), toFloat64(-2.1));
/* Int16 */
SELECT round(toInt16(13), toUInt8(2)), ceil(toInt16(13), toUInt8(2)), floor(toInt16(13), toUInt8(2));
SELECT round(toInt16(13), toUInt16(2)), ceil(toInt16(13), toUInt16(2)), floor(toInt16(13), toUInt16(2));
SELECT round(toInt16(13), toUInt32(2)), ceil(toInt16(13), toUInt32(2)), floor(toInt16(13), toUInt32(2));
SELECT round(toInt16(13), toUInt64(2)), ceil(toInt16(13), toUInt64(2)), floor(toInt16(13), toUInt64(2));
SELECT round(toInt16(13), toInt8(2)), ceil(toInt16(13), toInt8(2)), floor(toInt16(13), toInt8(2));
SELECT round(toInt16(13), toInt16(2)), ceil(toInt16(13), toInt16(2)), floor(toInt16(13), toInt16(2));
SELECT round(toInt16(13), toInt32(2)), ceil(toInt16(13), toInt32(2)), floor(toInt16(13), toInt32(2));
SELECT round(toInt16(13), toInt64(2)), ceil(toInt16(13), toInt64(2)), floor(toInt16(13), toInt64(2));
SELECT round(toInt16(13), toFloat32(2.1)), ceil(toInt16(13), toFloat32(2.1)), floor(toInt16(13), toFloat32(2.1));
SELECT round(toInt16(13), toFloat64(2.1)), ceil(toInt16(13), toFloat64(2.1)), floor(toInt16(13), toFloat64(2.1));
SELECT round(toInt16(13), toUInt8(1)), ceil(toInt16(13), toUInt8(1)), floor(toInt16(13), toUInt8(1));
SELECT round(toInt16(13), toUInt16(1)), ceil(toInt16(13), toUInt16(1)), floor(toInt16(13), toUInt16(1));
SELECT round(toInt16(13), toUInt32(1)), ceil(toInt16(13), toUInt32(1)), floor(toInt16(13), toUInt32(1));
SELECT round(toInt16(13), toUInt64(1)), ceil(toInt16(13), toUInt64(1)), floor(toInt16(13), toUInt64(1));
SELECT round(toInt16(13), toInt8(1)), ceil(toInt16(13), toInt8(1)), floor(toInt16(13), toInt8(1));
SELECT round(toInt16(13), toInt16(1)), ceil(toInt16(13), toInt16(1)), floor(toInt16(13), toInt16(1));
SELECT round(toInt16(13), toInt32(1)), ceil(toInt16(13), toInt32(1)), floor(toInt16(13), toInt32(1));
SELECT round(toInt16(13), toInt64(1)), ceil(toInt16(13), toInt64(1)), floor(toInt16(13), toInt64(1));
SELECT round(toInt16(13), toFloat32(1.1)), ceil(toInt16(13), toFloat32(1.1)), floor(toInt16(13), toFloat32(1.1));
SELECT round(toInt16(13), toFloat64(1.1)), ceil(toInt16(13), toFloat64(1.1)), floor(toInt16(13), toFloat64(1.1));
SELECT round(toInt16(13), toUInt16(0)), ceil(toInt16(13), toUInt16(0)), floor(toInt16(13), toUInt16(0));
SELECT round(toInt16(13), toUInt32(0)), ceil(toInt16(13), toUInt32(0)), floor(toInt16(13), toUInt32(0));
SELECT round(toInt16(13), toUInt64(0)), ceil(toInt16(13), toUInt64(0)), floor(toInt16(13), toUInt64(0));
SELECT round(toInt16(13), toInt8(0)), ceil(toInt16(13), toInt8(0)), floor(toInt16(13), toInt8(0));
SELECT round(toInt16(13), toInt16(0)), ceil(toInt16(13), toInt16(0)), floor(toInt16(13), toInt16(0));
SELECT round(toInt16(13), toInt32(0)), ceil(toInt16(13), toInt32(0)), floor(toInt16(13), toInt32(0));
SELECT round(toInt16(13), toInt64(0)), ceil(toInt16(13), toInt64(0)), floor(toInt16(13), toInt64(0));
SELECT round(toInt16(13), toFloat32(0.1)), ceil(toInt16(13), toFloat32(0.1)), floor(toInt16(13), toFloat32(0.1));
SELECT round(toInt16(13), toFloat64(0.1)), ceil(toInt16(13), toFloat64(0.1)), floor(toInt16(13), toFloat64(0.1));
SELECT round(toInt16(13), toInt8(-1)), ceil(toInt16(13), toInt8(-1)), floor(toInt16(13), toInt8(-1));
SELECT round(toInt16(13), toInt16(-1)), ceil(toInt16(13), toInt16(-1)), floor(toInt16(13), toInt16(-1));
SELECT round(toInt16(13), toInt32(-1)), ceil(toInt16(13), toInt32(-1)), floor(toInt16(13), toInt32(-1));
SELECT round(toInt16(13), toInt64(-1)), ceil(toInt16(13), toInt64(-1)), floor(toInt16(13), toInt64(-1));
SELECT round(toInt16(13), toFloat32(1.1)), ceil(toInt16(13), toFloat32(-1.1)), floor(toInt16(13), toFloat32(-1.1));
SELECT round(toInt16(13), toFloat64(1.1)), ceil(toInt16(13), toFloat64(-1.1)), floor(toInt16(13), toFloat64(-1.1));
SELECT round(toInt16(13), toInt8(-2)), ceil(toInt16(13), toInt8(-2)), floor(toInt16(13), toInt8(-2));
SELECT round(toInt16(13), toInt16(-2)), ceil(toInt16(13), toInt16(-2)), floor(toInt16(13), toInt16(-2));
SELECT round(toInt16(13), toInt32(-2)), ceil(toInt16(13), toInt32(-2)), floor(toInt16(13), toInt32(-2));
SELECT round(toInt16(13), toInt64(-2)), ceil(toInt16(13), toInt64(-2)), floor(toInt16(13), toInt64(-2));
SELECT round(toInt16(13), toFloat32(-2.1)), ceil(toInt16(13), toFloat32(-2.1)), floor(toInt16(13), toFloat32(-2.1));
SELECT round(toInt16(13), toFloat64(-2.1)), ceil(toInt16(13), toFloat64(-2.1)), floor(toInt16(13), toFloat64(-2.1));
/* Int32 */
SELECT round(toInt32(13), toUInt8(2)), ceil(toInt32(13), toUInt8(2)), floor(toInt32(13), toUInt8(2));
SELECT round(toInt32(13), toUInt16(2)), ceil(toInt32(13), toUInt16(2)), floor(toInt32(13), toUInt16(2));
SELECT round(toInt32(13), toUInt32(2)), ceil(toInt32(13), toUInt32(2)), floor(toInt32(13), toUInt32(2));
SELECT round(toInt32(13), toUInt64(2)), ceil(toInt32(13), toUInt64(2)), floor(toInt32(13), toUInt64(2));
SELECT round(toInt32(13), toInt8(2)), ceil(toInt32(13), toInt8(2)), floor(toInt32(13), toInt8(2));
SELECT round(toInt32(13), toInt16(2)), ceil(toInt32(13), toInt16(2)), floor(toInt32(13), toInt16(2));
SELECT round(toInt32(13), toInt32(2)), ceil(toInt32(13), toInt32(2)), floor(toInt32(13), toInt32(2));
SELECT round(toInt32(13), toInt64(2)), ceil(toInt32(13), toInt64(2)), floor(toInt32(13), toInt64(2));
SELECT round(toInt32(13), toFloat32(2.1)), ceil(toInt32(13), toFloat32(2.1)), floor(toInt32(13), toFloat32(2.1));
SELECT round(toInt32(13), toFloat64(2.1)), ceil(toInt32(13), toFloat64(2.1)), floor(toInt32(13), toFloat64(2.1));
SELECT round(toInt32(13), toUInt8(1)), ceil(toInt32(13), toUInt8(1)), floor(toInt32(13), toUInt8(1));
SELECT round(toInt32(13), toUInt16(1)), ceil(toInt32(13), toUInt16(1)), floor(toInt32(13), toUInt16(1));
SELECT round(toInt32(13), toUInt32(1)), ceil(toInt32(13), toUInt32(1)), floor(toInt32(13), toUInt32(1));
SELECT round(toInt32(13), toUInt64(1)), ceil(toInt32(13), toUInt64(1)), floor(toInt32(13), toUInt64(1));
SELECT round(toInt32(13), toInt8(1)), ceil(toInt32(13), toInt8(1)), floor(toInt32(13), toInt8(1));
SELECT round(toInt32(13), toInt16(1)), ceil(toInt32(13), toInt16(1)), floor(toInt32(13), toInt16(1));
SELECT round(toInt32(13), toInt32(1)), ceil(toInt32(13), toInt32(1)), floor(toInt32(13), toInt32(1));
SELECT round(toInt32(13), toInt64(1)), ceil(toInt32(13), toInt64(1)), floor(toInt32(13), toInt64(1));
SELECT round(toInt32(13), toFloat32(1.1)), ceil(toInt32(13), toFloat32(1.1)), floor(toInt32(13), toFloat32(1.1));
SELECT round(toInt32(13), toFloat64(1.1)), ceil(toInt32(13), toFloat64(1.1)), floor(toInt32(13), toFloat64(1.1));
SELECT round(toInt32(13), toUInt16(0)), ceil(toInt32(13), toUInt16(0)), floor(toInt32(13), toUInt16(0));
SELECT round(toInt32(13), toUInt32(0)), ceil(toInt32(13), toUInt32(0)), floor(toInt32(13), toUInt32(0));
SELECT round(toInt32(13), toUInt64(0)), ceil(toInt32(13), toUInt64(0)), floor(toInt32(13), toUInt64(0));
SELECT round(toInt32(13), toInt8(0)), ceil(toInt32(13), toInt8(0)), floor(toInt32(13), toInt8(0));
SELECT round(toInt32(13), toInt16(0)), ceil(toInt32(13), toInt16(0)), floor(toInt32(13), toInt16(0));
SELECT round(toInt32(13), toInt32(0)), ceil(toInt32(13), toInt32(0)), floor(toInt32(13), toInt32(0));
SELECT round(toInt32(13), toInt64(0)), ceil(toInt32(13), toInt64(0)), floor(toInt32(13), toInt64(0));
SELECT round(toInt32(13), toFloat32(0.1)), ceil(toInt32(13), toFloat32(0.1)), floor(toInt32(13), toFloat32(0.1));
SELECT round(toInt32(13), toFloat64(0.1)), ceil(toInt32(13), toFloat64(0.1)), floor(toInt32(13), toFloat64(0.1));
SELECT round(toInt32(13), toInt8(-1)), ceil(toInt32(13), toInt8(-1)), floor(toInt32(13), toInt8(-1));
SELECT round(toInt32(13), toInt16(-1)), ceil(toInt32(13), toInt16(-1)), floor(toInt32(13), toInt16(-1));
SELECT round(toInt32(13), toInt32(-1)), ceil(toInt32(13), toInt32(-1)), floor(toInt32(13), toInt32(-1));
SELECT round(toInt32(13), toInt64(-1)), ceil(toInt32(13), toInt64(-1)), floor(toInt32(13), toInt64(-1));
SELECT round(toInt32(13), toFloat32(1.1)), ceil(toInt32(13), toFloat32(-1.1)), floor(toInt32(13), toFloat32(-1.1));
SELECT round(toInt32(13), toFloat64(1.1)), ceil(toInt32(13), toFloat64(-1.1)), floor(toInt32(13), toFloat64(-1.1));
SELECT round(toInt32(13), toInt8(-2)), ceil(toInt32(13), toInt8(-2)), floor(toInt32(13), toInt8(-2));
SELECT round(toInt32(13), toInt16(-2)), ceil(toInt32(13), toInt16(-2)), floor(toInt32(13), toInt16(-2));
SELECT round(toInt32(13), toInt32(-2)), ceil(toInt32(13), toInt32(-2)), floor(toInt32(13), toInt32(-2));
SELECT round(toInt32(13), toInt64(-2)), ceil(toInt32(13), toInt64(-2)), floor(toInt32(13), toInt64(-2));
SELECT round(toInt32(13), toFloat32(-2.1)), ceil(toInt32(13), toFloat32(-2.1)), floor(toInt32(13), toFloat32(-2.1));
SELECT round(toInt32(13), toFloat64(-2.1)), ceil(toInt32(13), toFloat64(-2.1)), floor(toInt32(13), toFloat64(-2.1));
/* Int64 */
SELECT round(toInt64(13), toUInt8(2)), ceil(toInt64(13), toUInt8(2)), floor(toInt64(13), toUInt8(2));
SELECT round(toInt64(13), toUInt16(2)), ceil(toInt64(13), toUInt16(2)), floor(toInt64(13), toUInt16(2));
SELECT round(toInt64(13), toUInt32(2)), ceil(toInt64(13), toUInt32(2)), floor(toInt64(13), toUInt32(2));
SELECT round(toInt64(13), toUInt64(2)), ceil(toInt64(13), toUInt64(2)), floor(toInt64(13), toUInt64(2));
SELECT round(toInt64(13), toInt8(2)), ceil(toInt64(13), toInt8(2)), floor(toInt64(13), toInt8(2));
SELECT round(toInt64(13), toInt16(2)), ceil(toInt64(13), toInt16(2)), floor(toInt64(13), toInt16(2));
SELECT round(toInt64(13), toInt32(2)), ceil(toInt64(13), toInt32(2)), floor(toInt64(13), toInt32(2));
SELECT round(toInt64(13), toInt64(2)), ceil(toInt64(13), toInt64(2)), floor(toInt64(13), toInt64(2));
SELECT round(toInt64(13), toFloat32(2.1)), ceil(toInt64(13), toFloat32(2.1)), floor(toInt64(13), toFloat32(2.1));
SELECT round(toInt64(13), toFloat64(2.1)), ceil(toInt64(13), toFloat64(2.1)), floor(toInt64(13), toFloat64(2.1));
SELECT round(toInt64(13), toUInt8(1)), ceil(toInt64(13), toUInt8(1)), floor(toInt64(13), toUInt8(1));
SELECT round(toInt64(13), toUInt16(1)), ceil(toInt64(13), toUInt16(1)), floor(toInt64(13), toUInt16(1));
SELECT round(toInt64(13), toUInt32(1)), ceil(toInt64(13), toUInt32(1)), floor(toInt64(13), toUInt32(1));
SELECT round(toInt64(13), toUInt64(1)), ceil(toInt64(13), toUInt64(1)), floor(toInt64(13), toUInt64(1));
SELECT round(toInt64(13), toInt8(1)), ceil(toInt64(13), toInt8(1)), floor(toInt64(13), toInt8(1));
SELECT round(toInt64(13), toInt16(1)), ceil(toInt64(13), toInt16(1)), floor(toInt64(13), toInt16(1));
SELECT round(toInt64(13), toInt32(1)), ceil(toInt64(13), toInt32(1)), floor(toInt64(13), toInt32(1));
SELECT round(toInt64(13), toInt64(1)), ceil(toInt64(13), toInt64(1)), floor(toInt64(13), toInt64(1));
SELECT round(toInt64(13), toFloat32(1.1)), ceil(toInt64(13), toFloat32(1.1)), floor(toInt64(13), toFloat32(1.1));
SELECT round(toInt64(13), toFloat64(1.1)), ceil(toInt64(13), toFloat64(1.1)), floor(toInt64(13), toFloat64(1.1));
SELECT round(toInt64(13), toUInt16(0)), ceil(toInt64(13), toUInt16(0)), floor(toInt64(13), toUInt16(0));
SELECT round(toInt64(13), toUInt32(0)), ceil(toInt64(13), toUInt32(0)), floor(toInt64(13), toUInt32(0));
SELECT round(toInt64(13), toUInt64(0)), ceil(toInt64(13), toUInt64(0)), floor(toInt64(13), toUInt64(0));
SELECT round(toInt64(13), toInt8(0)), ceil(toInt64(13), toInt8(0)), floor(toInt64(13), toInt8(0));
SELECT round(toInt64(13), toInt16(0)), ceil(toInt64(13), toInt16(0)), floor(toInt64(13), toInt16(0));
SELECT round(toInt64(13), toInt32(0)), ceil(toInt64(13), toInt32(0)), floor(toInt64(13), toInt32(0));
SELECT round(toInt64(13), toInt64(0)), ceil(toInt64(13), toInt64(0)), floor(toInt64(13), toInt64(0));
SELECT round(toInt64(13), toFloat32(0.1)), ceil(toInt64(13), toFloat32(0.1)), floor(toInt64(13), toFloat32(0.1));
SELECT round(toInt64(13), toFloat64(0.1)), ceil(toInt64(13), toFloat64(0.1)), floor(toInt64(13), toFloat64(0.1));
SELECT round(toInt64(13), toInt8(-1)), ceil(toInt64(13), toInt8(-1)), floor(toInt64(13), toInt8(-1));
SELECT round(toInt64(13), toInt16(-1)), ceil(toInt64(13), toInt16(-1)), floor(toInt64(13), toInt16(-1));
SELECT round(toInt64(13), toInt32(-1)), ceil(toInt64(13), toInt32(-1)), floor(toInt64(13), toInt32(-1));
SELECT round(toInt64(13), toInt64(-1)), ceil(toInt64(13), toInt64(-1)), floor(toInt64(13), toInt64(-1));
SELECT round(toInt64(13), toFloat32(1.1)), ceil(toInt64(13), toFloat32(-1.1)), floor(toInt64(13), toFloat32(-1.1));
SELECT round(toInt64(13), toFloat64(1.1)), ceil(toInt64(13), toFloat64(-1.1)), floor(toInt64(13), toFloat64(-1.1));
SELECT round(toInt64(13), toInt8(-2)), ceil(toInt64(13), toInt8(-2)), floor(toInt64(13), toInt8(-2));
SELECT round(toInt64(13), toInt16(-2)), ceil(toInt64(13), toInt16(-2)), floor(toInt64(13), toInt16(-2));
SELECT round(toInt64(13), toInt32(-2)), ceil(toInt64(13), toInt32(-2)), floor(toInt64(13), toInt32(-2));
SELECT round(toInt64(13), toInt64(-2)), ceil(toInt64(13), toInt64(-2)), floor(toInt64(13), toInt64(-2));
SELECT round(toInt64(13), toFloat32(-2.1)), ceil(toInt64(13), toFloat32(-2.1)), floor(toInt64(13), toFloat32(-2.1));
SELECT round(toInt64(13), toFloat64(-2.1)), ceil(toInt64(13), toFloat64(-2.1)), floor(toInt64(13), toFloat64(-2.1));
/* Float32 */
SELECT round(toFloat32(13), toUInt8(2)), ceil(toFloat32(13), toUInt8(2)), floor(toFloat32(13), toUInt8(2));
SELECT round(toFloat32(13), toUInt16(2)), ceil(toFloat32(13), toUInt16(2)), floor(toFloat32(13), toUInt16(2));
SELECT round(toFloat32(13), toUInt32(2)), ceil(toFloat32(13), toUInt32(2)), floor(toFloat32(13), toUInt32(2));
SELECT round(toFloat32(13), toUInt64(2)), ceil(toFloat32(13), toUInt64(2)), floor(toFloat32(13), toUInt64(2));
SELECT round(toFloat32(13), toInt8(2)), ceil(toFloat32(13), toInt8(2)), floor(toFloat32(13), toInt8(2));
SELECT round(toFloat32(13), toInt16(2)), ceil(toFloat32(13), toInt16(2)), floor(toFloat32(13), toInt16(2));
SELECT round(toFloat32(13), toInt32(2)), ceil(toFloat32(13), toInt32(2)), floor(toFloat32(13), toInt32(2));
SELECT round(toFloat32(13), toInt64(2)), ceil(toFloat32(13), toInt64(2)), floor(toFloat32(13), toInt64(2));
SELECT round(toFloat32(13), toFloat32(2.1)), ceil(toFloat32(13), toFloat32(2.1)), floor(toFloat32(13), toFloat32(2.1));
SELECT round(toFloat32(13), toFloat64(2.1)), ceil(toFloat32(13), toFloat64(2.1)), floor(toFloat32(13), toFloat64(2.1));
SELECT round(toFloat32(13), toUInt8(1)), ceil(toFloat32(13), toUInt8(1)), floor(toFloat32(13), toUInt8(1));
SELECT round(toFloat32(13), toUInt16(1)), ceil(toFloat32(13), toUInt16(1)), floor(toFloat32(13), toUInt16(1));
SELECT round(toFloat32(13), toUInt32(1)), ceil(toFloat32(13), toUInt32(1)), floor(toFloat32(13), toUInt32(1));
SELECT round(toFloat32(13), toUInt64(1)), ceil(toFloat32(13), toUInt64(1)), floor(toFloat32(13), toUInt64(1));
SELECT round(toFloat32(13), toInt8(1)), ceil(toFloat32(13), toInt8(1)), floor(toFloat32(13), toInt8(1));
SELECT round(toFloat32(13), toInt16(1)), ceil(toFloat32(13), toInt16(1)), floor(toFloat32(13), toInt16(1));
SELECT round(toFloat32(13), toInt32(1)), ceil(toFloat32(13), toInt32(1)), floor(toFloat32(13), toInt32(1));
SELECT round(toFloat32(13), toInt64(1)), ceil(toFloat32(13), toInt64(1)), floor(toFloat32(13), toInt64(1));
SELECT round(toFloat32(13), toFloat32(1.1)), ceil(toFloat32(13), toFloat32(1.1)), floor(toFloat32(13), toFloat32(1.1));
SELECT round(toFloat32(13), toFloat64(1.1)), ceil(toFloat32(13), toFloat64(1.1)), floor(toFloat32(13), toFloat64(1.1));
SELECT round(toFloat32(13), toUInt16(0)), ceil(toFloat32(13), toUInt16(0)), floor(toFloat32(13), toUInt16(0));
SELECT round(toFloat32(13), toUInt32(0)), ceil(toFloat32(13), toUInt32(0)), floor(toFloat32(13), toUInt32(0));
SELECT round(toFloat32(13), toUInt64(0)), ceil(toFloat32(13), toUInt64(0)), floor(toFloat32(13), toUInt64(0));
SELECT round(toFloat32(13), toInt8(0)), ceil(toFloat32(13), toInt8(0)), floor(toFloat32(13), toInt8(0));
SELECT round(toFloat32(13), toInt16(0)), ceil(toFloat32(13), toInt16(0)), floor(toFloat32(13), toInt16(0));
SELECT round(toFloat32(13), toInt32(0)), ceil(toFloat32(13), toInt32(0)), floor(toFloat32(13), toInt32(0));
SELECT round(toFloat32(13), toInt64(0)), ceil(toFloat32(13), toInt64(0)), floor(toFloat32(13), toInt64(0));
SELECT round(toFloat32(13), toFloat32(0.1)), ceil(toFloat32(13), toFloat32(0.1)), floor(toFloat32(13), toFloat32(0.1));
SELECT round(toFloat32(13), toFloat64(0.1)), ceil(toFloat32(13), toFloat64(0.1)), floor(toFloat32(13), toFloat64(0.1));
SELECT round(toFloat32(13), toInt8(-1)), ceil(toFloat32(13), toInt8(-1)), floor(toFloat32(13), toInt8(-1));
SELECT round(toFloat32(13), toInt16(-1)), ceil(toFloat32(13), toInt16(-1)), floor(toFloat32(13), toInt16(-1));
SELECT round(toFloat32(13), toInt32(-1)), ceil(toFloat32(13), toInt32(-1)), floor(toFloat32(13), toInt32(-1));
SELECT round(toFloat32(13), toInt64(-1)), ceil(toFloat32(13), toInt64(-1)), floor(toFloat32(13), toInt64(-1));
SELECT round(toFloat32(13), toFloat32(1.1)), ceil(toFloat32(13), toFloat32(-1.1)), floor(toFloat32(13), toFloat32(-1.1));
SELECT round(toFloat32(13), toFloat64(1.1)), ceil(toFloat32(13), toFloat64(-1.1)), floor(toFloat32(13), toFloat64(-1.1));
SELECT round(toFloat32(13), toInt8(-2)), ceil(toFloat32(13), toInt8(-2)), floor(toFloat32(13), toInt8(-2));
SELECT round(toFloat32(13), toInt16(-2)), ceil(toFloat32(13), toInt16(-2)), floor(toFloat32(13), toInt16(-2));
SELECT round(toFloat32(13), toInt32(-2)), ceil(toFloat32(13), toInt32(-2)), floor(toFloat32(13), toInt32(-2));
SELECT round(toFloat32(13), toInt64(-2)), ceil(toFloat32(13), toInt64(-2)), floor(toFloat32(13), toInt64(-2));
SELECT round(toFloat32(13), toFloat32(-2.1)), ceil(toFloat32(13), toFloat32(-2.1)), floor(toFloat32(13), toFloat32(-2.1));
SELECT round(toFloat32(13), toFloat64(-2.1)), ceil(toFloat32(13), toFloat64(-2.1)), floor(toFloat32(13), toFloat64(-2.1));
/* Float64 */
SELECT round(toFloat64(13), toUInt8(2)), ceil(toFloat64(13), toUInt8(2)), floor(toFloat64(13), toUInt8(2));
SELECT round(toFloat64(13), toUInt16(2)), ceil(toFloat64(13), toUInt16(2)), floor(toFloat64(13), toUInt16(2));
SELECT round(toFloat64(13), toUInt32(2)), ceil(toFloat64(13), toUInt32(2)), floor(toFloat64(13), toUInt32(2));
SELECT round(toFloat64(13), toUInt64(2)), ceil(toFloat64(13), toUInt64(2)), floor(toFloat64(13), toUInt64(2));
SELECT round(toFloat64(13), toInt8(2)), ceil(toFloat64(13), toInt8(2)), floor(toFloat64(13), toInt8(2));
SELECT round(toFloat64(13), toInt16(2)), ceil(toFloat64(13), toInt16(2)), floor(toFloat64(13), toInt16(2));
SELECT round(toFloat64(13), toInt32(2)), ceil(toFloat64(13), toInt32(2)), floor(toFloat64(13), toInt32(2));
SELECT round(toFloat64(13), toInt64(2)), ceil(toFloat64(13), toInt64(2)), floor(toFloat64(13), toInt64(2));
SELECT round(toFloat64(13), toFloat32(2.1)), ceil(toFloat64(13), toFloat32(2.1)), floor(toFloat64(13), toFloat32(2.1));
SELECT round(toFloat64(13), toFloat64(2.1)), ceil(toFloat64(13), toFloat64(2.1)), floor(toFloat64(13), toFloat64(2.1));
SELECT round(toFloat64(13), toUInt8(1)), ceil(toFloat64(13), toUInt8(1)), floor(toFloat64(13), toUInt8(1));
SELECT round(toFloat64(13), toUInt16(1)), ceil(toFloat64(13), toUInt16(1)), floor(toFloat64(13), toUInt16(1));
SELECT round(toFloat64(13), toUInt32(1)), ceil(toFloat64(13), toUInt32(1)), floor(toFloat64(13), toUInt32(1));
SELECT round(toFloat64(13), toUInt64(1)), ceil(toFloat64(13), toUInt64(1)), floor(toFloat64(13), toUInt64(1));
SELECT round(toFloat64(13), toInt8(1)), ceil(toFloat64(13), toInt8(1)), floor(toFloat64(13), toInt8(1));
SELECT round(toFloat64(13), toInt16(1)), ceil(toFloat64(13), toInt16(1)), floor(toFloat64(13), toInt16(1));
SELECT round(toFloat64(13), toInt32(1)), ceil(toFloat64(13), toInt32(1)), floor(toFloat64(13), toInt32(1));
SELECT round(toFloat64(13), toInt64(1)), ceil(toFloat64(13), toInt64(1)), floor(toFloat64(13), toInt64(1));
SELECT round(toFloat64(13), toFloat32(1.1)), ceil(toFloat64(13), toFloat32(1.1)), floor(toFloat64(13), toFloat32(1.1));
SELECT round(toFloat64(13), toFloat64(1.1)), ceil(toFloat64(13), toFloat64(1.1)), floor(toFloat64(13), toFloat64(1.1));
SELECT round(toFloat64(13), toUInt16(0)), ceil(toFloat64(13), toUInt16(0)), floor(toFloat64(13), toUInt16(0));
SELECT round(toFloat64(13), toUInt32(0)), ceil(toFloat64(13), toUInt32(0)), floor(toFloat64(13), toUInt32(0));
SELECT round(toFloat64(13), toUInt64(0)), ceil(toFloat64(13), toUInt64(0)), floor(toFloat64(13), toUInt64(0));
SELECT round(toFloat64(13), toInt8(0)), ceil(toFloat64(13), toInt8(0)), floor(toFloat64(13), toInt8(0));
SELECT round(toFloat64(13), toInt16(0)), ceil(toFloat64(13), toInt16(0)), floor(toFloat64(13), toInt16(0));
SELECT round(toFloat64(13), toInt32(0)), ceil(toFloat64(13), toInt32(0)), floor(toFloat64(13), toInt32(0));
SELECT round(toFloat64(13), toInt64(0)), ceil(toFloat64(13), toInt64(0)), floor(toFloat64(13), toInt64(0));
SELECT round(toFloat64(13), toFloat32(0.1)), ceil(toFloat64(13), toFloat32(0.1)), floor(toFloat64(13), toFloat32(0.1));
SELECT round(toFloat64(13), toFloat64(0.1)), ceil(toFloat64(13), toFloat64(0.1)), floor(toFloat64(13), toFloat64(0.1));
SELECT round(toFloat64(13), toInt8(-1)), ceil(toFloat64(13), toInt8(-1)), floor(toFloat64(13), toInt8(-1));
SELECT round(toFloat64(13), toInt16(-1)), ceil(toFloat64(13), toInt16(-1)), floor(toFloat64(13), toInt16(-1));
SELECT round(toFloat64(13), toInt32(-1)), ceil(toFloat64(13), toInt32(-1)), floor(toFloat64(13), toInt32(-1));
SELECT round(toFloat64(13), toInt64(-1)), ceil(toFloat64(13), toInt64(-1)), floor(toFloat64(13), toInt64(-1));
SELECT round(toFloat64(13), toFloat32(1.1)), ceil(toFloat64(13), toFloat32(-1.1)), floor(toFloat64(13), toFloat32(-1.1));
SELECT round(toFloat64(13), toFloat64(1.1)), ceil(toFloat64(13), toFloat64(-1.1)), floor(toFloat64(13), toFloat64(-1.1));
SELECT round(toFloat64(13), toInt8(-2)), ceil(toFloat64(13), toInt8(-2)), floor(toFloat64(13), toInt8(-2));
SELECT round(toFloat64(13), toInt16(-2)), ceil(toFloat64(13), toInt16(-2)), floor(toFloat64(13), toInt16(-2));
SELECT round(toFloat64(13), toInt32(-2)), ceil(toFloat64(13), toInt32(-2)), floor(toFloat64(13), toInt32(-2));
SELECT round(toFloat64(13), toInt64(-2)), ceil(toFloat64(13), toInt64(-2)), floor(toFloat64(13), toInt64(-2));
SELECT round(toFloat64(13), toFloat32(-2.1)), ceil(toFloat64(13), toFloat32(-2.1)), floor(toFloat64(13), toFloat32(-2.1));
SELECT round(toFloat64(13), toFloat64(-2.1)), ceil(toFloat64(13), toFloat64(-2.1)), floor(toFloat64(13), toFloat64(-2.1));
/* Отрицательное значение */
/* Int8 */
SELECT round(toInt8(-13), toUInt8(2)), ceil(toInt8(-13), toUInt8(2)), floor(toInt8(-13), toUInt8(2));
SELECT round(toInt8(-13), toUInt16(2)), ceil(toInt8(-13), toUInt16(2)), floor(toInt8(-13), toUInt16(2));
SELECT round(toInt8(-13), toUInt32(2)), ceil(toInt8(-13), toUInt32(2)), floor(toInt8(-13), toUInt32(2));
SELECT round(toInt8(-13), toUInt64(2)), ceil(toInt8(-13), toUInt64(2)), floor(toInt8(-13), toUInt64(2));
SELECT round(toInt8(-13), toInt8(2)), ceil(toInt8(-13), toInt8(2)), floor(toInt8(-13), toInt8(2));
SELECT round(toInt8(-13), toInt16(2)), ceil(toInt8(-13), toInt16(2)), floor(toInt8(-13), toInt16(2));
SELECT round(toInt8(-13), toInt32(2)), ceil(toInt8(-13), toInt32(2)), floor(toInt8(-13), toInt32(2));
SELECT round(toInt8(-13), toInt64(2)), ceil(toInt8(-13), toInt64(2)), floor(toInt8(-13), toInt64(2));
SELECT round(toInt8(-13), toFloat32(2.1)), ceil(toInt8(-13), toFloat32(2.1)), floor(toInt8(-13), toFloat32(2.1));
SELECT round(toInt8(-13), toFloat64(2.1)), ceil(toInt8(-13), toFloat64(2.1)), floor(toInt8(-13), toFloat64(2.1));
SELECT round(toInt8(-13), toUInt8(1)), ceil(toInt8(-13), toUInt8(1)), floor(toInt8(-13), toUInt8(1));
SELECT round(toInt8(-13), toUInt16(1)), ceil(toInt8(-13), toUInt16(1)), floor(toInt8(-13), toUInt16(1));
SELECT round(toInt8(-13), toUInt32(1)), ceil(toInt8(-13), toUInt32(1)), floor(toInt8(-13), toUInt32(1));
SELECT round(toInt8(-13), toUInt64(1)), ceil(toInt8(-13), toUInt64(1)), floor(toInt8(-13), toUInt64(1));
SELECT round(toInt8(-13), toInt8(1)), ceil(toInt8(-13), toInt8(1)), floor(toInt8(-13), toInt8(1));
SELECT round(toInt8(-13), toInt16(1)), ceil(toInt8(-13), toInt16(1)), floor(toInt8(-13), toInt16(1));
SELECT round(toInt8(-13), toInt32(1)), ceil(toInt8(-13), toInt32(1)), floor(toInt8(-13), toInt32(1));
SELECT round(toInt8(-13), toInt64(1)), ceil(toInt8(-13), toInt64(1)), floor(toInt8(-13), toInt64(1));
SELECT round(toInt8(-13), toFloat32(1.1)), ceil(toInt8(-13), toFloat32(1.1)), floor(toInt8(-13), toFloat32(1.1));
SELECT round(toInt8(-13), toFloat64(1.1)), ceil(toInt8(-13), toFloat64(1.1)), floor(toInt8(-13), toFloat64(1.1));
SELECT round(toInt8(-13), toUInt16(0)), ceil(toInt8(-13), toUInt16(0)), floor(toInt8(-13), toUInt16(0));
SELECT round(toInt8(-13), toUInt32(0)), ceil(toInt8(-13), toUInt32(0)), floor(toInt8(-13), toUInt32(0));
SELECT round(toInt8(-13), toUInt64(0)), ceil(toInt8(-13), toUInt64(0)), floor(toInt8(-13), toUInt64(0));
SELECT round(toInt8(-13), toInt8(0)), ceil(toInt8(-13), toInt8(0)), floor(toInt8(-13), toInt8(0));
SELECT round(toInt8(-13), toInt16(0)), ceil(toInt8(-13), toInt16(0)), floor(toInt8(-13), toInt16(0));
SELECT round(toInt8(-13), toInt32(0)), ceil(toInt8(-13), toInt32(0)), floor(toInt8(-13), toInt32(0));
SELECT round(toInt8(-13), toInt64(0)), ceil(toInt8(-13), toInt64(0)), floor(toInt8(-13), toInt64(0));
SELECT round(toInt8(-13), toFloat32(0.1)), ceil(toInt8(-13), toFloat32(0.1)), floor(toInt8(-13), toFloat32(0.1));
SELECT round(toInt8(-13), toFloat64(0.1)), ceil(toInt8(-13), toFloat64(0.1)), floor(toInt8(-13), toFloat64(0.1));
SELECT round(toInt8(-13), toInt8(-1)), ceil(toInt8(-13), toInt8(-1)), floor(toInt8(-13), toInt8(-1));
SELECT round(toInt8(-13), toInt16(-1)), ceil(toInt8(-13), toInt16(-1)), floor(toInt8(-13), toInt16(-1));
SELECT round(toInt8(-13), toInt32(-1)), ceil(toInt8(-13), toInt32(-1)), floor(toInt8(-13), toInt32(-1));
SELECT round(toInt8(-13), toInt64(-1)), ceil(toInt8(-13), toInt64(-1)), floor(toInt8(-13), toInt64(-1));
SELECT round(toInt8(-13), toFloat32(1.1)), ceil(toInt8(-13), toFloat32(-1.1)), floor(toInt8(-13), toFloat32(-1.1));
SELECT round(toInt8(-13), toFloat64(1.1)), ceil(toInt8(-13), toFloat64(-1.1)), floor(toInt8(-13), toFloat64(-1.1));
SELECT round(toInt8(-13), toInt8(-2)), ceil(toInt8(-13), toInt8(-2)), floor(toInt8(-13), toInt8(-2));
SELECT round(toInt8(-13), toInt16(-2)), ceil(toInt8(-13), toInt16(-2)), floor(toInt8(-13), toInt16(-2));
SELECT round(toInt8(-13), toInt32(-2)), ceil(toInt8(-13), toInt32(-2)), floor(toInt8(-13), toInt32(-2));
SELECT round(toInt8(-13), toInt64(-2)), ceil(toInt8(-13), toInt64(-2)), floor(toInt8(-13), toInt64(-2));
SELECT round(toInt8(-13), toFloat32(-2.1)), ceil(toInt8(-13), toFloat32(-2.1)), floor(toInt8(-13), toFloat32(-2.1));
SELECT round(toInt8(-13), toFloat64(-2.1)), ceil(toInt8(-13), toFloat64(-2.1)), floor(toInt8(-13), toFloat64(-2.1));
/* Int16 */
SELECT round(toInt16(-13), toUInt8(2)), ceil(toInt16(-13), toUInt8(2)), floor(toInt16(-13), toUInt8(2));
SELECT round(toInt16(-13), toUInt16(2)), ceil(toInt16(-13), toUInt16(2)), floor(toInt16(-13), toUInt16(2));
SELECT round(toInt16(-13), toUInt32(2)), ceil(toInt16(-13), toUInt32(2)), floor(toInt16(-13), toUInt32(2));
SELECT round(toInt16(-13), toUInt64(2)), ceil(toInt16(-13), toUInt64(2)), floor(toInt16(-13), toUInt64(2));
SELECT round(toInt16(-13), toInt8(2)), ceil(toInt16(-13), toInt8(2)), floor(toInt16(-13), toInt8(2));
SELECT round(toInt16(-13), toInt16(2)), ceil(toInt16(-13), toInt16(2)), floor(toInt16(-13), toInt16(2));
SELECT round(toInt16(-13), toInt32(2)), ceil(toInt16(-13), toInt32(2)), floor(toInt16(-13), toInt32(2));
SELECT round(toInt16(-13), toInt64(2)), ceil(toInt16(-13), toInt64(2)), floor(toInt16(-13), toInt64(2));
SELECT round(toInt16(-13), toFloat32(2.1)), ceil(toInt16(-13), toFloat32(2.1)), floor(toInt16(-13), toFloat32(2.1));
SELECT round(toInt16(-13), toFloat64(2.1)), ceil(toInt16(-13), toFloat64(2.1)), floor(toInt16(-13), toFloat64(2.1));
SELECT round(toInt16(-13), toUInt8(1)), ceil(toInt16(-13), toUInt8(1)), floor(toInt16(-13), toUInt8(1));
SELECT round(toInt16(-13), toUInt16(1)), ceil(toInt16(-13), toUInt16(1)), floor(toInt16(-13), toUInt16(1));
SELECT round(toInt16(-13), toUInt32(1)), ceil(toInt16(-13), toUInt32(1)), floor(toInt16(-13), toUInt32(1));
SELECT round(toInt16(-13), toUInt64(1)), ceil(toInt16(-13), toUInt64(1)), floor(toInt16(-13), toUInt64(1));
SELECT round(toInt16(-13), toInt8(1)), ceil(toInt16(-13), toInt8(1)), floor(toInt16(-13), toInt8(1));
SELECT round(toInt16(-13), toInt16(1)), ceil(toInt16(-13), toInt16(1)), floor(toInt16(-13), toInt16(1));
SELECT round(toInt16(-13), toInt32(1)), ceil(toInt16(-13), toInt32(1)), floor(toInt16(-13), toInt32(1));
SELECT round(toInt16(-13), toInt64(1)), ceil(toInt16(-13), toInt64(1)), floor(toInt16(-13), toInt64(1));
SELECT round(toInt16(-13), toFloat32(1.1)), ceil(toInt16(-13), toFloat32(1.1)), floor(toInt16(-13), toFloat32(1.1));
SELECT round(toInt16(-13), toFloat64(1.1)), ceil(toInt16(-13), toFloat64(1.1)), floor(toInt16(-13), toFloat64(1.1));
SELECT round(toInt16(-13), toUInt16(0)), ceil(toInt16(-13), toUInt16(0)), floor(toInt16(-13), toUInt16(0));
SELECT round(toInt16(-13), toUInt32(0)), ceil(toInt16(-13), toUInt32(0)), floor(toInt16(-13), toUInt32(0));
SELECT round(toInt16(-13), toUInt64(0)), ceil(toInt16(-13), toUInt64(0)), floor(toInt16(-13), toUInt64(0));
SELECT round(toInt16(-13), toInt8(0)), ceil(toInt16(-13), toInt8(0)), floor(toInt16(-13), toInt8(0));
SELECT round(toInt16(-13), toInt16(0)), ceil(toInt16(-13), toInt16(0)), floor(toInt16(-13), toInt16(0));
SELECT round(toInt16(-13), toInt32(0)), ceil(toInt16(-13), toInt32(0)), floor(toInt16(-13), toInt32(0));
SELECT round(toInt16(-13), toInt64(0)), ceil(toInt16(-13), toInt64(0)), floor(toInt16(-13), toInt64(0));
SELECT round(toInt16(-13), toFloat32(0.1)), ceil(toInt16(-13), toFloat32(0.1)), floor(toInt16(-13), toFloat32(0.1));
SELECT round(toInt16(-13), toFloat64(0.1)), ceil(toInt16(-13), toFloat64(0.1)), floor(toInt16(-13), toFloat64(0.1));
SELECT round(toInt16(-13), toInt8(-1)), ceil(toInt16(-13), toInt8(-1)), floor(toInt16(-13), toInt8(-1));
SELECT round(toInt16(-13), toInt16(-1)), ceil(toInt16(-13), toInt16(-1)), floor(toInt16(-13), toInt16(-1));
SELECT round(toInt16(-13), toInt32(-1)), ceil(toInt16(-13), toInt32(-1)), floor(toInt16(-13), toInt32(-1));
SELECT round(toInt16(-13), toInt64(-1)), ceil(toInt16(-13), toInt64(-1)), floor(toInt16(-13), toInt64(-1));
SELECT round(toInt16(-13), toFloat32(1.1)), ceil(toInt16(-13), toFloat32(-1.1)), floor(toInt16(-13), toFloat32(-1.1));
SELECT round(toInt16(-13), toFloat64(1.1)), ceil(toInt16(-13), toFloat64(-1.1)), floor(toInt16(-13), toFloat64(-1.1));
SELECT round(toInt16(-13), toInt8(-2)), ceil(toInt16(-13), toInt8(-2)), floor(toInt16(-13), toInt8(-2));
SELECT round(toInt16(-13), toInt16(-2)), ceil(toInt16(-13), toInt16(-2)), floor(toInt16(-13), toInt16(-2));
SELECT round(toInt16(-13), toInt32(-2)), ceil(toInt16(-13), toInt32(-2)), floor(toInt16(-13), toInt32(-2));
SELECT round(toInt16(-13), toInt64(-2)), ceil(toInt16(-13), toInt64(-2)), floor(toInt16(-13), toInt64(-2));
SELECT round(toInt16(-13), toFloat32(-2.1)), ceil(toInt16(-13), toFloat32(-2.1)), floor(toInt16(-13), toFloat32(-2.1));
SELECT round(toInt16(-13), toFloat64(-2.1)), ceil(toInt16(-13), toFloat64(-2.1)), floor(toInt16(-13), toFloat64(-2.1));
/* Int32 */
SELECT round(toInt32(-13), toUInt8(2)), ceil(toInt32(-13), toUInt8(2)), floor(toInt32(-13), toUInt8(2));
SELECT round(toInt32(-13), toUInt16(2)), ceil(toInt32(-13), toUInt16(2)), floor(toInt32(-13), toUInt16(2));
SELECT round(toInt32(-13), toUInt32(2)), ceil(toInt32(-13), toUInt32(2)), floor(toInt32(-13), toUInt32(2));
SELECT round(toInt32(-13), toUInt64(2)), ceil(toInt32(-13), toUInt64(2)), floor(toInt32(-13), toUInt64(2));
SELECT round(toInt32(-13), toInt8(2)), ceil(toInt32(-13), toInt8(2)), floor(toInt32(-13), toInt8(2));
SELECT round(toInt32(-13), toInt16(2)), ceil(toInt32(-13), toInt16(2)), floor(toInt32(-13), toInt16(2));
SELECT round(toInt32(-13), toInt32(2)), ceil(toInt32(-13), toInt32(2)), floor(toInt32(-13), toInt32(2));
SELECT round(toInt32(-13), toInt64(2)), ceil(toInt32(-13), toInt64(2)), floor(toInt32(-13), toInt64(2));
SELECT round(toInt32(-13), toFloat32(2.1)), ceil(toInt32(-13), toFloat32(2.1)), floor(toInt32(-13), toFloat32(2.1));
SELECT round(toInt32(-13), toFloat64(2.1)), ceil(toInt32(-13), toFloat64(2.1)), floor(toInt32(-13), toFloat64(2.1));
SELECT round(toInt32(-13), toUInt8(1)), ceil(toInt32(-13), toUInt8(1)), floor(toInt32(-13), toUInt8(1));
SELECT round(toInt32(-13), toUInt16(1)), ceil(toInt32(-13), toUInt16(1)), floor(toInt32(-13), toUInt16(1));
SELECT round(toInt32(-13), toUInt32(1)), ceil(toInt32(-13), toUInt32(1)), floor(toInt32(-13), toUInt32(1));
SELECT round(toInt32(-13), toUInt64(1)), ceil(toInt32(-13), toUInt64(1)), floor(toInt32(-13), toUInt64(1));
SELECT round(toInt32(-13), toInt8(1)), ceil(toInt32(-13), toInt8(1)), floor(toInt32(-13), toInt8(1));
SELECT round(toInt32(-13), toInt16(1)), ceil(toInt32(-13), toInt16(1)), floor(toInt32(-13), toInt16(1));
SELECT round(toInt32(-13), toInt32(1)), ceil(toInt32(-13), toInt32(1)), floor(toInt32(-13), toInt32(1));
SELECT round(toInt32(-13), toInt64(1)), ceil(toInt32(-13), toInt64(1)), floor(toInt32(-13), toInt64(1));
SELECT round(toInt32(-13), toFloat32(1.1)), ceil(toInt32(-13), toFloat32(1.1)), floor(toInt32(-13), toFloat32(1.1));
SELECT round(toInt32(-13), toFloat64(1.1)), ceil(toInt32(-13), toFloat64(1.1)), floor(toInt32(-13), toFloat64(1.1));
SELECT round(toInt32(-13), toUInt16(0)), ceil(toInt32(-13), toUInt16(0)), floor(toInt32(-13), toUInt16(0));
SELECT round(toInt32(-13), toUInt32(0)), ceil(toInt32(-13), toUInt32(0)), floor(toInt32(-13), toUInt32(0));
SELECT round(toInt32(-13), toUInt64(0)), ceil(toInt32(-13), toUInt64(0)), floor(toInt32(-13), toUInt64(0));
SELECT round(toInt32(-13), toInt8(0)), ceil(toInt32(-13), toInt8(0)), floor(toInt32(-13), toInt8(0));
SELECT round(toInt32(-13), toInt16(0)), ceil(toInt32(-13), toInt16(0)), floor(toInt32(-13), toInt16(0));
SELECT round(toInt32(-13), toInt32(0)), ceil(toInt32(-13), toInt32(0)), floor(toInt32(-13), toInt32(0));
SELECT round(toInt32(-13), toInt64(0)), ceil(toInt32(-13), toInt64(0)), floor(toInt32(-13), toInt64(0));
SELECT round(toInt32(-13), toFloat32(0.1)), ceil(toInt32(-13), toFloat32(0.1)), floor(toInt32(-13), toFloat32(0.1));
SELECT round(toInt32(-13), toFloat64(0.1)), ceil(toInt32(-13), toFloat64(0.1)), floor(toInt32(-13), toFloat64(0.1));
SELECT round(toInt32(-13), toInt8(-1)), ceil(toInt32(-13), toInt8(-1)), floor(toInt32(-13), toInt8(-1));
SELECT round(toInt32(-13), toInt16(-1)), ceil(toInt32(-13), toInt16(-1)), floor(toInt32(-13), toInt16(-1));
SELECT round(toInt32(-13), toInt32(-1)), ceil(toInt32(-13), toInt32(-1)), floor(toInt32(-13), toInt32(-1));
SELECT round(toInt32(-13), toInt64(-1)), ceil(toInt32(-13), toInt64(-1)), floor(toInt32(-13), toInt64(-1));
SELECT round(toInt32(-13), toFloat32(1.1)), ceil(toInt32(-13), toFloat32(-1.1)), floor(toInt32(-13), toFloat32(-1.1));
SELECT round(toInt32(-13), toFloat64(1.1)), ceil(toInt32(-13), toFloat64(-1.1)), floor(toInt32(-13), toFloat64(-1.1));
SELECT round(toInt32(-13), toInt8(-2)), ceil(toInt32(-13), toInt8(-2)), floor(toInt32(-13), toInt8(-2));
SELECT round(toInt32(-13), toInt16(-2)), ceil(toInt32(-13), toInt16(-2)), floor(toInt32(-13), toInt16(-2));
SELECT round(toInt32(-13), toInt32(-2)), ceil(toInt32(-13), toInt32(-2)), floor(toInt32(-13), toInt32(-2));
SELECT round(toInt32(-13), toInt64(-2)), ceil(toInt32(-13), toInt64(-2)), floor(toInt32(-13), toInt64(-2));
SELECT round(toInt32(-13), toFloat32(-2.1)), ceil(toInt32(-13), toFloat32(-2.1)), floor(toInt32(-13), toFloat32(-2.1));
SELECT round(toInt32(-13), toFloat64(-2.1)), ceil(toInt32(-13), toFloat64(-2.1)), floor(toInt32(-13), toFloat64(-2.1));
/* Int64 */
SELECT round(toInt64(-13), toUInt8(2)), ceil(toInt64(-13), toUInt8(2)), floor(toInt64(-13), toUInt8(2));
SELECT round(toInt64(-13), toUInt16(2)), ceil(toInt64(-13), toUInt16(2)), floor(toInt64(-13), toUInt16(2));
SELECT round(toInt64(-13), toUInt32(2)), ceil(toInt64(-13), toUInt32(2)), floor(toInt64(-13), toUInt32(2));
SELECT round(toInt64(-13), toUInt64(2)), ceil(toInt64(-13), toUInt64(2)), floor(toInt64(-13), toUInt64(2));
SELECT round(toInt64(-13), toInt8(2)), ceil(toInt64(-13), toInt8(2)), floor(toInt64(-13), toInt8(2));
SELECT round(toInt64(-13), toInt16(2)), ceil(toInt64(-13), toInt16(2)), floor(toInt64(-13), toInt16(2));
SELECT round(toInt64(-13), toInt32(2)), ceil(toInt64(-13), toInt32(2)), floor(toInt64(-13), toInt32(2));
SELECT round(toInt64(-13), toInt64(2)), ceil(toInt64(-13), toInt64(2)), floor(toInt64(-13), toInt64(2));
SELECT round(toInt64(-13), toFloat32(2.1)), ceil(toInt64(-13), toFloat32(2.1)), floor(toInt64(-13), toFloat32(2.1));
SELECT round(toInt64(-13), toFloat64(2.1)), ceil(toInt64(-13), toFloat64(2.1)), floor(toInt64(-13), toFloat64(2.1));
SELECT round(toInt64(-13), toUInt8(1)), ceil(toInt64(-13), toUInt8(1)), floor(toInt64(-13), toUInt8(1));
SELECT round(toInt64(-13), toUInt16(1)), ceil(toInt64(-13), toUInt16(1)), floor(toInt64(-13), toUInt16(1));
SELECT round(toInt64(-13), toUInt32(1)), ceil(toInt64(-13), toUInt32(1)), floor(toInt64(-13), toUInt32(1));
SELECT round(toInt64(-13), toUInt64(1)), ceil(toInt64(-13), toUInt64(1)), floor(toInt64(-13), toUInt64(1));
SELECT round(toInt64(-13), toInt8(1)), ceil(toInt64(-13), toInt8(1)), floor(toInt64(-13), toInt8(1));
SELECT round(toInt64(-13), toInt16(1)), ceil(toInt64(-13), toInt16(1)), floor(toInt64(-13), toInt16(1));
SELECT round(toInt64(-13), toInt32(1)), ceil(toInt64(-13), toInt32(1)), floor(toInt64(-13), toInt32(1));
SELECT round(toInt64(-13), toInt64(1)), ceil(toInt64(-13), toInt64(1)), floor(toInt64(-13), toInt64(1));
SELECT round(toInt64(-13), toFloat32(1.1)), ceil(toInt64(-13), toFloat32(1.1)), floor(toInt64(-13), toFloat32(1.1));
SELECT round(toInt64(-13), toFloat64(1.1)), ceil(toInt64(-13), toFloat64(1.1)), floor(toInt64(-13), toFloat64(1.1));
SELECT round(toInt64(-13), toUInt16(0)), ceil(toInt64(-13), toUInt16(0)), floor(toInt64(-13), toUInt16(0));
SELECT round(toInt64(-13), toUInt32(0)), ceil(toInt64(-13), toUInt32(0)), floor(toInt64(-13), toUInt32(0));
SELECT round(toInt64(-13), toUInt64(0)), ceil(toInt64(-13), toUInt64(0)), floor(toInt64(-13), toUInt64(0));
SELECT round(toInt64(-13), toInt8(0)), ceil(toInt64(-13), toInt8(0)), floor(toInt64(-13), toInt8(0));
SELECT round(toInt64(-13), toInt16(0)), ceil(toInt64(-13), toInt16(0)), floor(toInt64(-13), toInt16(0));
SELECT round(toInt64(-13), toInt32(0)), ceil(toInt64(-13), toInt32(0)), floor(toInt64(-13), toInt32(0));
SELECT round(toInt64(-13), toInt64(0)), ceil(toInt64(-13), toInt64(0)), floor(toInt64(-13), toInt64(0));
SELECT round(toInt64(-13), toFloat32(0.1)), ceil(toInt64(-13), toFloat32(0.1)), floor(toInt64(-13), toFloat32(0.1));
SELECT round(toInt64(-13), toFloat64(0.1)), ceil(toInt64(-13), toFloat64(0.1)), floor(toInt64(-13), toFloat64(0.1));
SELECT round(toInt64(-13), toInt8(-1)), ceil(toInt64(-13), toInt8(-1)), floor(toInt64(-13), toInt8(-1));
SELECT round(toInt64(-13), toInt16(-1)), ceil(toInt64(-13), toInt16(-1)), floor(toInt64(-13), toInt16(-1));
SELECT round(toInt64(-13), toInt32(-1)), ceil(toInt64(-13), toInt32(-1)), floor(toInt64(-13), toInt32(-1));
SELECT round(toInt64(-13), toInt64(-1)), ceil(toInt64(-13), toInt64(-1)), floor(toInt64(-13), toInt64(-1));
SELECT round(toInt64(-13), toFloat32(1.1)), ceil(toInt64(-13), toFloat32(-1.1)), floor(toInt64(-13), toFloat32(-1.1));
SELECT round(toInt64(-13), toFloat64(1.1)), ceil(toInt64(-13), toFloat64(-1.1)), floor(toInt64(-13), toFloat64(-1.1));
SELECT round(toInt64(-13), toInt8(-2)), ceil(toInt64(-13), toInt8(-2)), floor(toInt64(-13), toInt8(-2));
SELECT round(toInt64(-13), toInt16(-2)), ceil(toInt64(-13), toInt16(-2)), floor(toInt64(-13), toInt16(-2));
SELECT round(toInt64(-13), toInt32(-2)), ceil(toInt64(-13), toInt32(-2)), floor(toInt64(-13), toInt32(-2));
SELECT round(toInt64(-13), toInt64(-2)), ceil(toInt64(-13), toInt64(-2)), floor(toInt64(-13), toInt64(-2));
SELECT round(toInt64(-13), toFloat32(-2.1)), ceil(toInt64(-13), toFloat32(-2.1)), floor(toInt64(-13), toFloat32(-2.1));
SELECT round(toInt64(-13), toFloat64(-2.1)), ceil(toInt64(-13), toFloat64(-2.1)), floor(toInt64(-13), toFloat64(-2.1));
/* Float32 */
SELECT round(toFloat32(-13), toUInt8(2)), ceil(toFloat32(-13), toUInt8(2)), floor(toFloat32(-13), toUInt8(2));
SELECT round(toFloat32(-13), toUInt16(2)), ceil(toFloat32(-13), toUInt16(2)), floor(toFloat32(-13), toUInt16(2));
SELECT round(toFloat32(-13), toUInt32(2)), ceil(toFloat32(-13), toUInt32(2)), floor(toFloat32(-13), toUInt32(2));
SELECT round(toFloat32(-13), toUInt64(2)), ceil(toFloat32(-13), toUInt64(2)), floor(toFloat32(-13), toUInt64(2));
SELECT round(toFloat32(-13), toInt8(2)), ceil(toFloat32(-13), toInt8(2)), floor(toFloat32(-13), toInt8(2));
SELECT round(toFloat32(-13), toInt16(2)), ceil(toFloat32(-13), toInt16(2)), floor(toFloat32(-13), toInt16(2));
SELECT round(toFloat32(-13), toInt32(2)), ceil(toFloat32(-13), toInt32(2)), floor(toFloat32(-13), toInt32(2));
SELECT round(toFloat32(-13), toInt64(2)), ceil(toFloat32(-13), toInt64(2)), floor(toFloat32(-13), toInt64(2));
SELECT round(toFloat32(-13), toFloat32(2.1)), ceil(toFloat32(-13), toFloat32(2.1)), floor(toFloat32(-13), toFloat32(2.1));
SELECT round(toFloat32(-13), toFloat64(2.1)), ceil(toFloat32(-13), toFloat64(2.1)), floor(toFloat32(-13), toFloat64(2.1));
SELECT round(toFloat32(-13), toUInt8(1)), ceil(toFloat32(-13), toUInt8(1)), floor(toFloat32(-13), toUInt8(1));
SELECT round(toFloat32(-13), toUInt16(1)), ceil(toFloat32(-13), toUInt16(1)), floor(toFloat32(-13), toUInt16(1));
SELECT round(toFloat32(-13), toUInt32(1)), ceil(toFloat32(-13), toUInt32(1)), floor(toFloat32(-13), toUInt32(1));
SELECT round(toFloat32(-13), toUInt64(1)), ceil(toFloat32(-13), toUInt64(1)), floor(toFloat32(-13), toUInt64(1));
SELECT round(toFloat32(-13), toInt8(1)), ceil(toFloat32(-13), toInt8(1)), floor(toFloat32(-13), toInt8(1));
SELECT round(toFloat32(-13), toInt16(1)), ceil(toFloat32(-13), toInt16(1)), floor(toFloat32(-13), toInt16(1));
SELECT round(toFloat32(-13), toInt32(1)), ceil(toFloat32(-13), toInt32(1)), floor(toFloat32(-13), toInt32(1));
SELECT round(toFloat32(-13), toInt64(1)), ceil(toFloat32(-13), toInt64(1)), floor(toFloat32(-13), toInt64(1));
SELECT round(toFloat32(-13), toFloat32(1.1)), ceil(toFloat32(-13), toFloat32(1.1)), floor(toFloat32(-13), toFloat32(1.1));
SELECT round(toFloat32(-13), toFloat64(1.1)), ceil(toFloat32(-13), toFloat64(1.1)), floor(toFloat32(-13), toFloat64(1.1));
SELECT round(toFloat32(-13), toUInt16(0)), ceil(toFloat32(-13), toUInt16(0)), floor(toFloat32(-13), toUInt16(0));
SELECT round(toFloat32(-13), toUInt32(0)), ceil(toFloat32(-13), toUInt32(0)), floor(toFloat32(-13), toUInt32(0));
SELECT round(toFloat32(-13), toUInt64(0)), ceil(toFloat32(-13), toUInt64(0)), floor(toFloat32(-13), toUInt64(0));
SELECT round(toFloat32(-13), toInt8(0)), ceil(toFloat32(-13), toInt8(0)), floor(toFloat32(-13), toInt8(0));
SELECT round(toFloat32(-13), toInt16(0)), ceil(toFloat32(-13), toInt16(0)), floor(toFloat32(-13), toInt16(0));
SELECT round(toFloat32(-13), toInt32(0)), ceil(toFloat32(-13), toInt32(0)), floor(toFloat32(-13), toInt32(0));
SELECT round(toFloat32(-13), toInt64(0)), ceil(toFloat32(-13), toInt64(0)), floor(toFloat32(-13), toInt64(0));
SELECT round(toFloat32(-13), toFloat32(0.1)), ceil(toFloat32(-13), toFloat32(0.1)), floor(toFloat32(-13), toFloat32(0.1));
SELECT round(toFloat32(-13), toFloat64(0.1)), ceil(toFloat32(-13), toFloat64(0.1)), floor(toFloat32(-13), toFloat64(0.1));
SELECT round(toFloat32(-13), toInt8(-1)), ceil(toFloat32(-13), toInt8(-1)), floor(toFloat32(-13), toInt8(-1));
SELECT round(toFloat32(-13), toInt16(-1)), ceil(toFloat32(-13), toInt16(-1)), floor(toFloat32(-13), toInt16(-1));
SELECT round(toFloat32(-13), toInt32(-1)), ceil(toFloat32(-13), toInt32(-1)), floor(toFloat32(-13), toInt32(-1));
SELECT round(toFloat32(-13), toInt64(-1)), ceil(toFloat32(-13), toInt64(-1)), floor(toFloat32(-13), toInt64(-1));
SELECT round(toFloat32(-13), toFloat32(1.1)), ceil(toFloat32(-13), toFloat32(-1.1)), floor(toFloat32(-13), toFloat32(-1.1));
SELECT round(toFloat32(-13), toFloat64(1.1)), ceil(toFloat32(-13), toFloat64(-1.1)), floor(toFloat32(-13), toFloat64(-1.1));
SELECT round(toFloat32(-13), toInt8(-2)), ceil(toFloat32(-13), toInt8(-2)), floor(toFloat32(-13), toInt8(-2));
SELECT round(toFloat32(-13), toInt16(-2)), ceil(toFloat32(-13), toInt16(-2)), floor(toFloat32(-13), toInt16(-2));
SELECT round(toFloat32(-13), toInt32(-2)), ceil(toFloat32(-13), toInt32(-2)), floor(toFloat32(-13), toInt32(-2));
SELECT round(toFloat32(-13), toInt64(-2)), ceil(toFloat32(-13), toInt64(-2)), floor(toFloat32(-13), toInt64(-2));
SELECT round(toFloat32(-13), toFloat32(-2.1)), ceil(toFloat32(-13), toFloat32(-2.1)), floor(toFloat32(-13), toFloat32(-2.1));
SELECT round(toFloat32(-13), toFloat64(-2.1)), ceil(toFloat32(-13), toFloat64(-2.1)), floor(toFloat32(-13), toFloat64(-2.1));
/* Float64 */
SELECT round(toFloat64(-13), toUInt8(2)), ceil(toFloat64(-13), toUInt8(2)), floor(toFloat64(-13), toUInt8(2));
SELECT round(toFloat64(-13), toUInt16(2)), ceil(toFloat64(-13), toUInt16(2)), floor(toFloat64(-13), toUInt16(2));
SELECT round(toFloat64(-13), toUInt32(2)), ceil(toFloat64(-13), toUInt32(2)), floor(toFloat64(-13), toUInt32(2));
SELECT round(toFloat64(-13), toUInt64(2)), ceil(toFloat64(-13), toUInt64(2)), floor(toFloat64(-13), toUInt64(2));
SELECT round(toFloat64(-13), toInt8(2)), ceil(toFloat64(-13), toInt8(2)), floor(toFloat64(-13), toInt8(2));
SELECT round(toFloat64(-13), toInt16(2)), ceil(toFloat64(-13), toInt16(2)), floor(toFloat64(-13), toInt16(2));
SELECT round(toFloat64(-13), toInt32(2)), ceil(toFloat64(-13), toInt32(2)), floor(toFloat64(-13), toInt32(2));
SELECT round(toFloat64(-13), toInt64(2)), ceil(toFloat64(-13), toInt64(2)), floor(toFloat64(-13), toInt64(2));
SELECT round(toFloat64(-13), toFloat32(2.1)), ceil(toFloat64(-13), toFloat32(2.1)), floor(toFloat64(-13), toFloat32(2.1));
SELECT round(toFloat64(-13), toFloat64(2.1)), ceil(toFloat64(-13), toFloat64(2.1)), floor(toFloat64(-13), toFloat64(2.1));
SELECT round(toFloat64(-13), toUInt8(1)), ceil(toFloat64(-13), toUInt8(1)), floor(toFloat64(-13), toUInt8(1));
SELECT round(toFloat64(-13), toUInt16(1)), ceil(toFloat64(-13), toUInt16(1)), floor(toFloat64(-13), toUInt16(1));
SELECT round(toFloat64(-13), toUInt32(1)), ceil(toFloat64(-13), toUInt32(1)), floor(toFloat64(-13), toUInt32(1));
SELECT round(toFloat64(-13), toUInt64(1)), ceil(toFloat64(-13), toUInt64(1)), floor(toFloat64(-13), toUInt64(1));
SELECT round(toFloat64(-13), toInt8(1)), ceil(toFloat64(-13), toInt8(1)), floor(toFloat64(-13), toInt8(1));
SELECT round(toFloat64(-13), toInt16(1)), ceil(toFloat64(-13), toInt16(1)), floor(toFloat64(-13), toInt16(1));
SELECT round(toFloat64(-13), toInt32(1)), ceil(toFloat64(-13), toInt32(1)), floor(toFloat64(-13), toInt32(1));
SELECT round(toFloat64(-13), toInt64(1)), ceil(toFloat64(-13), toInt64(1)), floor(toFloat64(-13), toInt64(1));
SELECT round(toFloat64(-13), toFloat32(1.1)), ceil(toFloat64(-13), toFloat32(1.1)), floor(toFloat64(-13), toFloat32(1.1));
SELECT round(toFloat64(-13), toFloat64(1.1)), ceil(toFloat64(-13), toFloat64(1.1)), floor(toFloat64(-13), toFloat64(1.1));
SELECT round(toFloat64(-13), toUInt16(0)), ceil(toFloat64(-13), toUInt16(0)), floor(toFloat64(-13), toUInt16(0));
SELECT round(toFloat64(-13), toUInt32(0)), ceil(toFloat64(-13), toUInt32(0)), floor(toFloat64(-13), toUInt32(0));
SELECT round(toFloat64(-13), toUInt64(0)), ceil(toFloat64(-13), toUInt64(0)), floor(toFloat64(-13), toUInt64(0));
SELECT round(toFloat64(-13), toInt8(0)), ceil(toFloat64(-13), toInt8(0)), floor(toFloat64(-13), toInt8(0));
SELECT round(toFloat64(-13), toInt16(0)), ceil(toFloat64(-13), toInt16(0)), floor(toFloat64(-13), toInt16(0));
SELECT round(toFloat64(-13), toInt32(0)), ceil(toFloat64(-13), toInt32(0)), floor(toFloat64(-13), toInt32(0));
SELECT round(toFloat64(-13), toInt64(0)), ceil(toFloat64(-13), toInt64(0)), floor(toFloat64(-13), toInt64(0));
SELECT round(toFloat64(-13), toFloat32(0.1)), ceil(toFloat64(-13), toFloat32(0.1)), floor(toFloat64(-13), toFloat32(0.1));
SELECT round(toFloat64(-13), toFloat64(0.1)), ceil(toFloat64(-13), toFloat64(0.1)), floor(toFloat64(-13), toFloat64(0.1));
SELECT round(toFloat64(-13), toInt8(-1)), ceil(toFloat64(-13), toInt8(-1)), floor(toFloat64(-13), toInt8(-1));
SELECT round(toFloat64(-13), toInt16(-1)), ceil(toFloat64(-13), toInt16(-1)), floor(toFloat64(-13), toInt16(-1));
SELECT round(toFloat64(-13), toInt32(-1)), ceil(toFloat64(-13), toInt32(-1)), floor(toFloat64(-13), toInt32(-1));
SELECT round(toFloat64(-13), toInt64(-1)), ceil(toFloat64(-13), toInt64(-1)), floor(toFloat64(-13), toInt64(-1));
SELECT round(toFloat64(-13), toFloat32(1.1)), ceil(toFloat64(-13), toFloat32(-1.1)), floor(toFloat64(-13), toFloat32(-1.1));
SELECT round(toFloat64(-13), toFloat64(1.1)), ceil(toFloat64(-13), toFloat64(-1.1)), floor(toFloat64(-13), toFloat64(-1.1));
SELECT round(toFloat64(-13), toInt8(-2)), ceil(toFloat64(-13), toInt8(-2)), floor(toFloat64(-13), toInt8(-2));
SELECT round(toFloat64(-13), toInt16(-2)), ceil(toFloat64(-13), toInt16(-2)), floor(toFloat64(-13), toInt16(-2));
SELECT round(toFloat64(-13), toInt32(-2)), ceil(toFloat64(-13), toInt32(-2)), floor(toFloat64(-13), toInt32(-2));
SELECT round(toFloat64(-13), toInt64(-2)), ceil(toFloat64(-13), toInt64(-2)), floor(toFloat64(-13), toInt64(-2));
SELECT round(toFloat64(-13), toFloat32(-2.1)), ceil(toFloat64(-13), toFloat32(-2.1)), floor(toFloat64(-13), toFloat32(-2.1));
SELECT round(toFloat64(-13), toFloat64(-2.1)), ceil(toFloat64(-13), toFloat64(-2.1)), floor(toFloat64(-13), toFloat64(-2.1));
/* Положительное число с плавающей точкой */
SELECT round(toFloat64(2.718281828459), toUInt8(2)), ceil(toFloat64(2.718281828459), toUInt8(2)), floor(toFloat64(2.718281828459), toUInt8(2));
SELECT round(toFloat64(2.718281828459), toUInt16(2)), ceil(toFloat64(2.718281828459), toUInt16(2)), floor(toFloat64(2.718281828459), toUInt16(2));
SELECT round(toFloat64(2.718281828459), toUInt32(2)), ceil(toFloat64(2.718281828459), toUInt32(2)), floor(toFloat64(2.718281828459), toUInt32(2));
SELECT round(toFloat64(2.718281828459), toUInt64(2)), ceil(toFloat64(2.718281828459), toUInt64(2)), floor(toFloat64(2.718281828459), toUInt64(2));
SELECT round(toFloat64(2.718281828459), toInt8(2)), ceil(toFloat64(2.718281828459), toInt8(2)), floor(toFloat64(2.718281828459), toInt8(2));
SELECT round(toFloat64(2.718281828459), toInt16(2)), ceil(toFloat64(2.718281828459), toInt16(2)), floor(toFloat64(2.718281828459), toInt16(2));
SELECT round(toFloat64(2.718281828459), toInt32(2)), ceil(toFloat64(2.718281828459), toInt32(2)), floor(toFloat64(2.718281828459), toInt32(2));
SELECT round(toFloat64(2.718281828459), toInt64(2)), ceil(toFloat64(2.718281828459), toInt64(2)), floor(toFloat64(2.718281828459), toInt64(2));
SELECT round(toFloat64(2.718281828459), toFloat32(2.1)), ceil(toFloat64(2.718281828459), toFloat32(2.1)), floor(toFloat64(2.718281828459), toFloat32(2.1));
SELECT round(toFloat64(2.718281828459), toFloat64(2.1)), ceil(toFloat64(2.718281828459), toFloat64(2.1)), floor(toFloat64(2.718281828459), toFloat64(2.1));
SELECT round(toFloat64(2.718281828459), toUInt8(1)), ceil(toFloat64(2.718281828459), toUInt8(1)), floor(toFloat64(2.718281828459), toUInt8(1));
SELECT round(toFloat64(2.718281828459), toUInt16(1)), ceil(toFloat64(2.718281828459), toUInt16(1)), floor(toFloat64(2.718281828459), toUInt16(1));
SELECT round(toFloat64(2.718281828459), toUInt32(1)), ceil(toFloat64(2.718281828459), toUInt32(1)), floor(toFloat64(2.718281828459), toUInt32(1));
SELECT round(toFloat64(2.718281828459), toUInt64(1)), ceil(toFloat64(2.718281828459), toUInt64(1)), floor(toFloat64(2.718281828459), toUInt64(1));
SELECT round(toFloat64(2.718281828459), toInt8(1)), ceil(toFloat64(2.718281828459), toInt8(1)), floor(toFloat64(2.718281828459), toInt8(1));
SELECT round(toFloat64(2.718281828459), toInt16(1)), ceil(toFloat64(2.718281828459), toInt16(1)), floor(toFloat64(2.718281828459), toInt16(1));
SELECT round(toFloat64(2.718281828459), toInt32(1)), ceil(toFloat64(2.718281828459), toInt32(1)), floor(toFloat64(2.718281828459), toInt32(1));
SELECT round(toFloat64(2.718281828459), toInt64(1)), ceil(toFloat64(2.718281828459), toInt64(1)), floor(toFloat64(2.718281828459), toInt64(1));
SELECT round(toFloat64(2.718281828459), toFloat32(1.1)), ceil(toFloat64(2.718281828459), toFloat32(1.1)), floor(toFloat64(2.718281828459), toFloat32(1.1));
SELECT round(toFloat64(2.718281828459), toFloat64(1.1)), ceil(toFloat64(2.718281828459), toFloat64(1.1)), floor(toFloat64(2.718281828459), toFloat64(1.1));
SELECT round(toFloat64(2.718281828459), toUInt16(0)), ceil(toFloat64(2.718281828459), toUInt16(0)), floor(toFloat64(2.718281828459), toUInt16(0));
SELECT round(toFloat64(2.718281828459), toUInt32(0)), ceil(toFloat64(2.718281828459), toUInt32(0)), floor(toFloat64(2.718281828459), toUInt32(0));
SELECT round(toFloat64(2.718281828459), toUInt64(0)), ceil(toFloat64(2.718281828459), toUInt64(0)), floor(toFloat64(2.718281828459), toUInt64(0));
SELECT round(toFloat64(2.718281828459), toInt8(0)), ceil(toFloat64(2.718281828459), toInt8(0)), floor(toFloat64(2.718281828459), toInt8(0));
SELECT round(toFloat64(2.718281828459), toInt16(0)), ceil(toFloat64(2.718281828459), toInt16(0)), floor(toFloat64(2.718281828459), toInt16(0));
SELECT round(toFloat64(2.718281828459), toInt32(0)), ceil(toFloat64(2.718281828459), toInt32(0)), floor(toFloat64(2.718281828459), toInt32(0));
SELECT round(toFloat64(2.718281828459), toInt64(0)), ceil(toFloat64(2.718281828459), toInt64(0)), floor(toFloat64(2.718281828459), toInt64(0));
SELECT round(toFloat64(2.718281828459), toFloat32(0.1)), ceil(toFloat64(2.718281828459), toFloat32(0.1)), floor(toFloat64(2.718281828459), toFloat32(0.1));
SELECT round(toFloat64(2.718281828459), toFloat64(0.1)), ceil(toFloat64(2.718281828459), toFloat64(0.1)), floor(toFloat64(2.718281828459), toFloat64(0.1));
SELECT round(toFloat64(2.718281828459), toInt8(-1)), ceil(toFloat64(2.718281828459), toInt8(-1)), floor(toFloat64(2.718281828459), toInt8(-1));
SELECT round(toFloat64(2.718281828459), toInt16(-1)), ceil(toFloat64(2.718281828459), toInt16(-1)), floor(toFloat64(2.718281828459), toInt16(-1));
SELECT round(toFloat64(2.718281828459), toInt32(-1)), ceil(toFloat64(2.718281828459), toInt32(-1)), floor(toFloat64(2.718281828459), toInt32(-1));
SELECT round(toFloat64(2.718281828459), toInt64(-1)), ceil(toFloat64(2.718281828459), toInt64(-1)), floor(toFloat64(2.718281828459), toInt64(-1));
SELECT round(toFloat64(2.718281828459), toFloat32(1.1)), ceil(toFloat64(2.718281828459), toFloat32(-1.1)), floor(toFloat64(2.718281828459), toFloat32(-1.1));
SELECT round(toFloat64(2.718281828459), toFloat64(1.1)), ceil(toFloat64(2.718281828459), toFloat64(-1.1)), floor(toFloat64(2.718281828459), toFloat64(-1.1));
SELECT round(toFloat64(2.718281828459), toInt8(-2)), ceil(toFloat64(2.718281828459), toInt8(-2)), floor(toFloat64(2.718281828459), toInt8(-2));
SELECT round(toFloat64(2.718281828459), toInt16(-2)), ceil(toFloat64(2.718281828459), toInt16(-2)), floor(toFloat64(2.718281828459), toInt16(-2));
SELECT round(toFloat64(2.718281828459), toInt32(-2)), ceil(toFloat64(2.718281828459), toInt32(-2)), floor(toFloat64(2.718281828459), toInt32(-2));
SELECT round(toFloat64(2.718281828459), toInt64(-2)), ceil(toFloat64(2.718281828459), toInt64(-2)), floor(toFloat64(2.718281828459), toInt64(-2));
SELECT round(toFloat64(2.718281828459), toFloat32(-2.1)), ceil(toFloat64(2.718281828459), toFloat32(-2.1)), floor(toFloat64(2.718281828459), toFloat32(-2.1));
SELECT round(toFloat64(2.718281828459), toFloat64(-2.1)), ceil(toFloat64(2.718281828459), toFloat64(-2.1)), floor(toFloat64(2.718281828459), toFloat64(-2.1));
/* Отрицательное число с плавающей точкой */
SELECT round(toFloat64(-2.718281828459), toUInt8(2)), ceil(toFloat64(-2.718281828459), toUInt8(2)), floor(toFloat64(-2.718281828459), toUInt8(2));
SELECT round(toFloat64(-2.718281828459), toUInt16(2)), ceil(toFloat64(-2.718281828459), toUInt16(2)), floor(toFloat64(-2.718281828459), toUInt16(2));
SELECT round(toFloat64(-2.718281828459), toUInt32(2)), ceil(toFloat64(-2.718281828459), toUInt32(2)), floor(toFloat64(-2.718281828459), toUInt32(2));
SELECT round(toFloat64(-2.718281828459), toUInt64(2)), ceil(toFloat64(-2.718281828459), toUInt64(2)), floor(toFloat64(-2.718281828459), toUInt64(2));
SELECT round(toFloat64(-2.718281828459), toInt8(2)), ceil(toFloat64(-2.718281828459), toInt8(2)), floor(toFloat64(-2.718281828459), toInt8(2));
SELECT round(toFloat64(-2.718281828459), toInt16(2)), ceil(toFloat64(-2.718281828459), toInt16(2)), floor(toFloat64(-2.718281828459), toInt16(2));
SELECT round(toFloat64(-2.718281828459), toInt32(2)), ceil(toFloat64(-2.718281828459), toInt32(2)), floor(toFloat64(-2.718281828459), toInt32(2));
SELECT round(toFloat64(-2.718281828459), toInt64(2)), ceil(toFloat64(-2.718281828459), toInt64(2)), floor(toFloat64(-2.718281828459), toInt64(2));
SELECT round(toFloat64(-2.718281828459), toFloat32(2.1)), ceil(toFloat64(-2.718281828459), toFloat32(2.1)), floor(toFloat64(-2.718281828459), toFloat32(2.1));
SELECT round(toFloat64(-2.718281828459), toFloat64(2.1)), ceil(toFloat64(-2.718281828459), toFloat64(2.1)), floor(toFloat64(-2.718281828459), toFloat64(2.1));
SELECT round(toFloat64(-2.718281828459), toUInt8(1)), ceil(toFloat64(-2.718281828459), toUInt8(1)), floor(toFloat64(-2.718281828459), toUInt8(1));
SELECT round(toFloat64(-2.718281828459), toUInt16(1)), ceil(toFloat64(-2.718281828459), toUInt16(1)), floor(toFloat64(-2.718281828459), toUInt16(1));
SELECT round(toFloat64(-2.718281828459), toUInt32(1)), ceil(toFloat64(-2.718281828459), toUInt32(1)), floor(toFloat64(-2.718281828459), toUInt32(1));
SELECT round(toFloat64(-2.718281828459), toUInt64(1)), ceil(toFloat64(-2.718281828459), toUInt64(1)), floor(toFloat64(-2.718281828459), toUInt64(1));
SELECT round(toFloat64(-2.718281828459), toInt8(1)), ceil(toFloat64(-2.718281828459), toInt8(1)), floor(toFloat64(-2.718281828459), toInt8(1));
SELECT round(toFloat64(-2.718281828459), toInt16(1)), ceil(toFloat64(-2.718281828459), toInt16(1)), floor(toFloat64(-2.718281828459), toInt16(1));
SELECT round(toFloat64(-2.718281828459), toInt32(1)), ceil(toFloat64(-2.718281828459), toInt32(1)), floor(toFloat64(-2.718281828459), toInt32(1));
SELECT round(toFloat64(-2.718281828459), toInt64(1)), ceil(toFloat64(-2.718281828459), toInt64(1)), floor(toFloat64(-2.718281828459), toInt64(1));
SELECT round(toFloat64(-2.718281828459), toFloat32(1.1)), ceil(toFloat64(-2.718281828459), toFloat32(1.1)), floor(toFloat64(-2.718281828459), toFloat32(1.1));
SELECT round(toFloat64(-2.718281828459), toFloat64(1.1)), ceil(toFloat64(-2.718281828459), toFloat64(1.1)), floor(toFloat64(-2.718281828459), toFloat64(1.1));
SELECT round(toFloat64(-2.718281828459), toUInt16(0)), ceil(toFloat64(-2.718281828459), toUInt16(0)), floor(toFloat64(-2.718281828459), toUInt16(0));
SELECT round(toFloat64(-2.718281828459), toUInt32(0)), ceil(toFloat64(-2.718281828459), toUInt32(0)), floor(toFloat64(-2.718281828459), toUInt32(0));
SELECT round(toFloat64(-2.718281828459), toUInt64(0)), ceil(toFloat64(-2.718281828459), toUInt64(0)), floor(toFloat64(-2.718281828459), toUInt64(0));
SELECT round(toFloat64(-2.718281828459), toInt8(0)), ceil(toFloat64(-2.718281828459), toInt8(0)), floor(toFloat64(-2.718281828459), toInt8(0));
SELECT round(toFloat64(-2.718281828459), toInt16(0)), ceil(toFloat64(-2.718281828459), toInt16(0)), floor(toFloat64(-2.718281828459), toInt16(0));
SELECT round(toFloat64(-2.718281828459), toInt32(0)), ceil(toFloat64(-2.718281828459), toInt32(0)), floor(toFloat64(-2.718281828459), toInt32(0));
SELECT round(toFloat64(-2.718281828459), toInt64(0)), ceil(toFloat64(-2.718281828459), toInt64(0)), floor(toFloat64(-2.718281828459), toInt64(0));
SELECT round(toFloat64(-2.718281828459), toFloat32(0.1)), ceil(toFloat64(-2.718281828459), toFloat32(0.1)), floor(toFloat64(-2.718281828459), toFloat32(0.1));
SELECT round(toFloat64(-2.718281828459), toFloat64(0.1)), ceil(toFloat64(-2.718281828459), toFloat64(0.1)), floor(toFloat64(-2.718281828459), toFloat64(0.1));
SELECT round(toFloat64(-2.718281828459), toInt8(-1)), ceil(toFloat64(-2.718281828459), toInt8(-1)), floor(toFloat64(-2.718281828459), toInt8(-1));
SELECT round(toFloat64(-2.718281828459), toInt16(-1)), ceil(toFloat64(-2.718281828459), toInt16(-1)), floor(toFloat64(-2.718281828459), toInt16(-1));
SELECT round(toFloat64(-2.718281828459), toInt32(-1)), ceil(toFloat64(-2.718281828459), toInt32(-1)), floor(toFloat64(-2.718281828459), toInt32(-1));
SELECT round(toFloat64(-2.718281828459), toInt64(-1)), ceil(toFloat64(-2.718281828459), toInt64(-1)), floor(toFloat64(-2.718281828459), toInt64(-1));
SELECT round(toFloat64(-2.718281828459), toFloat32(1.1)), ceil(toFloat64(-2.718281828459), toFloat32(-1.1)), floor(toFloat64(-2.718281828459), toFloat32(-1.1));
SELECT round(toFloat64(-2.718281828459), toFloat64(1.1)), ceil(toFloat64(-2.718281828459), toFloat64(-1.1)), floor(toFloat64(-2.718281828459), toFloat64(-1.1));
SELECT round(toFloat64(-2.718281828459), toInt8(-2)), ceil(toFloat64(-2.718281828459), toInt8(-2)), floor(toFloat64(-2.718281828459), toInt8(-2));
SELECT round(toFloat64(-2.718281828459), toInt16(-2)), ceil(toFloat64(-2.718281828459), toInt16(-2)), floor(toFloat64(-2.718281828459), toInt16(-2));
SELECT round(toFloat64(-2.718281828459), toInt32(-2)), ceil(toFloat64(-2.718281828459), toInt32(-2)), floor(toFloat64(-2.718281828459), toInt32(-2));
SELECT round(toFloat64(-2.718281828459), toInt64(-2)), ceil(toFloat64(-2.718281828459), toInt64(-2)), floor(toFloat64(-2.718281828459), toInt64(-2));
SELECT round(toFloat64(-2.718281828459), toFloat32(-2.1)), ceil(toFloat64(-2.718281828459), toFloat32(-2.1)), floor(toFloat64(-2.718281828459), toFloat32(-2.1));
SELECT round(toFloat64(-2.718281828459), toFloat64(-2.1)), ceil(toFloat64(-2.718281828459), toFloat64(-2.1)), floor(toFloat64(-2.718281828459), toFloat64(-2.1));
/* Misc. */
SELECT round(13112221, -1), ceil(13112221, -1), floor(13112221, -1);
SELECT round(13112221, -2), ceil(13112221, -2), floor(13112221, -2);
SELECT round(13112221, -3), ceil(13112221, -3), floor(13112221, -3);
SELECT round(13112221, -4), ceil(13112221, -4), floor(13112221, -4);
SELECT round(13112221, -5), ceil(13112221, -5), floor(13112221, -5);
SELECT round(13112221, -6), ceil(13112221, -6), floor(13112221, -6);
SELECT round(13112221, -7), ceil(13112221, -7), floor(13112221, -7);
SELECT round(13112221, -8), ceil(13112221, -8), floor(13112221, -8);
SELECT round(13112221, -9), ceil(13112221, -9), floor(13112221, -9);
SELECT round(13112221, -10), ceil(13112221, -10), floor(13112221, -10);
SELECT round(13112221, -11), ceil(13112221, -11), floor(13112221, -11);
SELECT round(13112221, -12), ceil(13112221, -12), floor(13112221, -12);
SELECT round(13112221, -13), ceil(13112221, -13), floor(13112221, -13);
SELECT round(13112221, -14), ceil(13112221, -14), floor(13112221, -14);
SELECT round(13112221, -15), ceil(13112221, -15), floor(13112221, -15);
SELECT round(13112221, -16), ceil(13112221, -16), floor(13112221, -16);
SELECT round(13112221, -17), ceil(13112221, -17), floor(13112221, -17);
SELECT round(13112221, -18), ceil(13112221, -18), floor(13112221, -18);
SELECT round(13112221, -19), ceil(13112221, -19), floor(13112221, -19);
SELECT round(13112221, -20), ceil(13112221, -20), floor(13112221, -20);
SELECT round(2.718281828459045, 1), ceil(2.718281828459045, 1), floor(2.718281828459045, 1);
SELECT round(2.718281828459045, 2), ceil(2.718281828459045, 2), floor(2.718281828459045, 2);
SELECT round(2.718281828459045, 3), ceil(2.718281828459045, 3), floor(2.718281828459045, 3);
SELECT round(2.718281828459045, 4), ceil(2.718281828459045, 4), floor(2.718281828459045, 4);
SELECT round(2.718281828459045, 5), ceil(2.718281828459045, 5), floor(2.718281828459045, 5);
SELECT round(2.718281828459045, 6), ceil(2.718281828459045, 6), floor(2.718281828459045, 6);
SELECT round(2.718281828459045, 7), ceil(2.718281828459045, 7), floor(2.718281828459045, 7);
SELECT round(2.718281828459045, 8), ceil(2.718281828459045, 8), floor(2.718281828459045, 8);
SELECT round(2.718281828459045, 9), ceil(2.718281828459045, 9), floor(2.718281828459045, 9);
SELECT round(2.718281828459045, 10), ceil(2.718281828459045, 10), floor(2.718281828459045, 10);
SELECT round(2.718281828459045, 11), ceil(2.718281828459045, 11), floor(2.718281828459045, 11);
SELECT round(2.718281828459045, 12), ceil(2.718281828459045, 12), floor(2.718281828459045, 12);
SELECT round(2.718281828459045, 13), ceil(2.718281828459045, 13), floor(2.718281828459045, 13);
SELECT round(2.718281828459045, 14), ceil(2.718281828459045, 14), floor(2.718281828459045, 14);
SELECT round(2.718281828459045, 15), ceil(2.718281828459045, 15), floor(2.718281828459045, 15);
SELECT round(2.718281828459045, 16), ceil(2.718281828459045, 16), floor(2.718281828459045, 16);
SELECT round(2.718281828459045, 17), ceil(2.718281828459045, 17), floor(2.718281828459045, 17);
SELECT round(2.718281828459045, 18), ceil(2.718281828459045, 18), floor(2.718281828459045, 18);
SELECT round(2.718281828459045, 19), ceil(2.718281828459045, 19), floor(2.718281828459045, 19);
SELECT round(2.718281828459045, 20), ceil(2.718281828459045, 20), floor(2.718281828459045, 20);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(1) AS x);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(2) AS x);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(3) AS x);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(4) AS x);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(5) AS x);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(6) AS x);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(7) AS x);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(8) AS x);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(9) AS x);
SELECT round(y,3) FROM (SELECT 2.718281828459045 + 1/(1+x*x) AS y ARRAY JOIN range(10) AS x);
/* Negative zeroes. */
SELECT round(-0.002);
SELECT round(-0.002, -1);
SELECT round(-0.002, 1);

View File

@ -0,0 +1,2 @@
2 6 Hello
2 6 Hello

Some files were not shown because too many files have changed in this diff Show More