mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge
This commit is contained in:
commit
e34a1fcbca
@ -1,90 +0,0 @@
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Сделано в целях отладки. Подлежит удалению.
|
||||
*/
|
||||
|
||||
struct AggregateFunctionDebugData
|
||||
{
|
||||
UInt32 value;
|
||||
|
||||
AggregateFunctionDebugData()
|
||||
{
|
||||
value = 0xAAAAAAAA;
|
||||
|
||||
if (rand() % 1000 == 0)
|
||||
throw Exception("Test1");
|
||||
}
|
||||
|
||||
~AggregateFunctionDebugData()
|
||||
{
|
||||
try
|
||||
{
|
||||
if (value == 0xDEADDEAD)
|
||||
throw Exception("Double free");
|
||||
|
||||
if (value != 0xAAAAAAAA)
|
||||
throw Exception("Corruption");
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
std::terminate();
|
||||
}
|
||||
|
||||
value = 0xDEADDEAD;
|
||||
}
|
||||
};
|
||||
|
||||
class AggregateFunctionDebug final : public IUnaryAggregateFunction<AggregateFunctionDebugData, AggregateFunctionDebug>
|
||||
{
|
||||
public:
|
||||
String getName() const { return "debug"; }
|
||||
|
||||
DataTypePtr getReturnType() const
|
||||
{
|
||||
return new DataTypeUInt32;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
}
|
||||
|
||||
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
|
||||
{
|
||||
if (rand() % 1000 == 0)
|
||||
throw Exception("Test2");
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
|
||||
{
|
||||
if (rand() % 1000 == 0)
|
||||
throw Exception("Test3");
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
|
||||
{
|
||||
if (rand() % 1000 == 0)
|
||||
throw Exception("Test4");
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
|
||||
{
|
||||
if (rand() % 1000 == 0)
|
||||
throw Exception("Test5");
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
|
||||
{
|
||||
if (rand() % 1000 == 0)
|
||||
throw Exception("Test6");
|
||||
|
||||
static_cast<ColumnUInt32 &>(to).getData().push_back(123);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -151,20 +151,18 @@ public:
|
||||
|
||||
void * realloc(void * buf, size_t old_size, size_t new_size)
|
||||
{
|
||||
/// Было в stack_memory, там и останется.
|
||||
if (new_size <= N)
|
||||
return buf;
|
||||
|
||||
/// Уже не помещалось в stack_memory.
|
||||
if (old_size > N)
|
||||
return HashTableAllocator::realloc(buf, old_size, new_size);
|
||||
|
||||
buf = ::malloc(new_size);
|
||||
if (nullptr == buf)
|
||||
DB::throwFromErrno("HashTableAllocator: Cannot malloc.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
|
||||
memcpy(buf, stack_memory, old_size);
|
||||
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
|
||||
|
||||
return buf;
|
||||
/// Было в stack_memory, но теперь не помещается.
|
||||
void * new_buf = HashTableAllocator::alloc(new_size);
|
||||
memcpy(new_buf, buf, old_size);
|
||||
return new_buf;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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(); }
|
||||
|
@ -51,6 +51,7 @@
|
||||
M(DelayedInserts) \
|
||||
M(RejectedInserts) \
|
||||
M(DelayedInsertsMilliseconds) \
|
||||
M(SynchronousMergeOnInsert) \
|
||||
\
|
||||
M(ZooKeeperInit) \
|
||||
M(ZooKeeperTransactions) \
|
||||
|
@ -12,7 +12,7 @@
|
||||
*/
|
||||
|
||||
#include <stdint.h>
|
||||
|
||||
#include <cstdlib>
|
||||
|
||||
#define ROTL(x,b) (u64)( ((x) << (b)) | ( (x) >> (64 - (b))) )
|
||||
|
||||
@ -76,6 +76,7 @@ public:
|
||||
v3 = 0x7465646279746573ULL ^ k1;
|
||||
|
||||
cnt = 0;
|
||||
current_word = 0;
|
||||
}
|
||||
|
||||
void update(const char * data, u64 size)
|
||||
|
8
dbms/include/DB/Common/getFQDNOrHostName.h
Normal file
8
dbms/include/DB/Common/getFQDNOrHostName.h
Normal file
@ -0,0 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <string>
|
||||
|
||||
/** Получить FQDN для локального сервера путём DNS-резолвинга hostname - аналогично вызову утилиты hostname с флагом -f.
|
||||
* Если не получилось отрезолвить, то вернуть hostname - аналогично вызову утилиты hostname без флагов или uname -n.
|
||||
*/
|
||||
const std::string & getFQDNOrHostName();
|
@ -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
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "AddingConstColumnBlockInputStream"; }
|
||||
String getName() const override { return "AddingConstColumn"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "AddingDefaultBlockInputStream"; }
|
||||
String getName() const override { return "AddingDefault"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "AggregatingBlockInputStream"; }
|
||||
String getName() const override { return "Aggregating"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "AggregatingSortedBlockInputStream"; }
|
||||
String getName() const override { return "AggregatingSorted"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
children.push_back(in_);
|
||||
}
|
||||
|
||||
String getName() const override { return "AsynchronousBlockInputStream"; }
|
||||
String getName() const override { return "Asynchronous"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ struct BlockStreamProfileInfo
|
||||
using BlockStreamProfileInfos = std::vector<const BlockStreamProfileInfo *>;
|
||||
BlockStreamProfileInfos nested_infos;
|
||||
|
||||
String column_names;
|
||||
|
||||
/// Собрать BlockStreamProfileInfo для ближайших в дереве источников с именем name. Пример; собрать все info для PartialSorting stream-ов.
|
||||
void collectInfosForStreamsWithName(const char * name, BlockStreamProfileInfos & res) const;
|
||||
|
||||
|
@ -23,7 +23,7 @@ public:
|
||||
|
||||
~CollapsingFinalBlockInputStream();
|
||||
|
||||
String getName() const override { return "CollapsingFinalBlockInputStream"; }
|
||||
String getName() const override { return "CollapsingFinal"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "CollapsingSortedBlockInputStream"; }
|
||||
String getName() const override { return "CollapsingSorted"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
current_stream = children.begin();
|
||||
}
|
||||
|
||||
String getName() const override { return "ConcatBlockInputStream"; }
|
||||
String getName() const override { return "Concat"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
children.push_back(input);
|
||||
}
|
||||
|
||||
String getName() const override { return "CreatingSetsBlockInputStream"; }
|
||||
String getName() const override { return "CreatingSets"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
/// Пустой columns_ значит все столбцы.
|
||||
DistinctBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_, Names columns_);
|
||||
|
||||
String getName() const override { return "DistinctBlockInputStream"; }
|
||||
String getName() const override { return "Distinct"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "ExpressionBlockInputStream"; }
|
||||
String getName() const override { return "Expression"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
FilterBlockInputStream(BlockInputStreamPtr input_, ssize_t filter_column_);
|
||||
FilterBlockInputStream(BlockInputStreamPtr input_, const String & filter_column_name_);
|
||||
|
||||
String getName() const override { return "FilterBlockInputStream"; }
|
||||
String getName() const override { return "Filter"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -51,7 +51,6 @@ public:
|
||||
/** Для вывода дерева преобразований потока данных (плана выполнения запроса).
|
||||
*/
|
||||
virtual String getName() const = 0;
|
||||
virtual String getShortName() const; /// То же самое, но без BlockInputStream на конце.
|
||||
|
||||
/** Уникальный идентификатор части конвейера выполнения запроса.
|
||||
* Источники с одинаковым идентификатором считаются идентичными
|
||||
|
@ -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;
|
||||
|
||||
|
||||
/** Смотрит за тем, как работает источник блоков.
|
||||
* Позволяет получить информацию для профайлинга:
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
LazyBlockInputStream(Generator generator_)
|
||||
: generator(generator_) {}
|
||||
|
||||
String getName() const override { return "LazyBlockInputStream"; }
|
||||
String getName() const override { return "Lazy"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -18,7 +18,7 @@ class LimitBlockInputStream : public IProfilingBlockInputStream
|
||||
public:
|
||||
LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_ = 0);
|
||||
|
||||
String getName() const override { return "LimitBlockInputStream"; }
|
||||
String getName() const override { return "Limit"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "MaterializingBlockInputStream"; }
|
||||
String getName() const override { return "Materializing"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
MergeSortingBlocksBlockInputStream(Blocks & blocks_, SortDescription & description_,
|
||||
size_t max_merged_block_size_, size_t limit_ = 0);
|
||||
|
||||
String getName() const override { return "MergeSortingBlocksBlockInputStream"; }
|
||||
String getName() const override { return "MergeSortingBlocks"; }
|
||||
String getID() const override { return getName(); }
|
||||
|
||||
protected:
|
||||
@ -66,14 +66,14 @@ 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_);
|
||||
}
|
||||
|
||||
String getName() const override { return "MergeSortingBlockInputStream"; }
|
||||
String getName() const override { return "MergeSorting"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
@ -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;
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "MergingAggregatedBlockInputStream"; }
|
||||
String getName() const override { return "MergingAggregated"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
children.insert(children.end(), inputs_.begin(), inputs_.end());
|
||||
}
|
||||
|
||||
String getName() const override { return "MergingSortedBlockInputStream"; }
|
||||
String getName() const override { return "MergingSorted"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataTypes/DataTypeFactory.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
|
||||
@ -16,10 +15,10 @@ 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"; }
|
||||
String getName() const override { return "Native"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
@ -35,7 +34,6 @@ protected:
|
||||
|
||||
private:
|
||||
ReadBuffer & istr;
|
||||
const DataTypeFactory & data_type_factory;
|
||||
UInt64 server_revision;
|
||||
};
|
||||
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "NullAndDoCopyBlockInputStream"; }
|
||||
String getName() const override { return "NullAndDoCopy"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -12,7 +12,7 @@ class NullBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
Block read() override { return Block(); }
|
||||
String getName() const override { return "NullBlockInputStream"; }
|
||||
String getName() const override { return "Null"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -18,7 +18,7 @@ class OneBlockInputStream : public IProfilingBlockInputStream
|
||||
public:
|
||||
OneBlockInputStream(const Block & block_) : block(block_) {}
|
||||
|
||||
String getName() const override { return "OneBlockInputStream"; }
|
||||
String getName() const override { return "One"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
children.insert(children.end(), inputs.begin(), inputs.end());
|
||||
}
|
||||
|
||||
String getName() const override { return "ParallelAggregatingBlockInputStream"; }
|
||||
String getName() const override { return "ParallelAggregating"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -21,7 +21,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "PartialSortingBlockInputStream"; }
|
||||
String getName() const override { return "PartialSorting"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -61,7 +61,7 @@ public:
|
||||
}
|
||||
|
||||
|
||||
String getName() const override { return "RemoteBlockInputStream"; }
|
||||
String getName() const override { return "Remote"; }
|
||||
|
||||
|
||||
String getID() const override
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "RemoveColumnsBlockInputStream"; }
|
||||
String getName() const override { return "RemoveColumns"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "SummingSortedBlockInputStream"; }
|
||||
String getName() const override { return "SummingSorted"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
@ -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)
|
||||
{
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "TotalsHavingBlockInputStream"; }
|
||||
String getName() const override { return "TotalsHaving"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -36,7 +36,7 @@ public:
|
||||
children = inputs;
|
||||
}
|
||||
|
||||
String getName() const override { return "UnionBlockInputStream"; }
|
||||
String getName() const override { return "Union"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -13,7 +13,7 @@ using Poco::SharedPtr;
|
||||
/** Тип - состояние агрегатной функции.
|
||||
* Параметры типа - это агрегатная функция, типы её аргументов и её параметры (для параметрических агрегатных функций).
|
||||
*/
|
||||
class DataTypeAggregateFunction : public IDataType
|
||||
class DataTypeAggregateFunction final : public IDataType
|
||||
{
|
||||
private:
|
||||
AggregateFunctionPtr function;
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeArray : public IDataType
|
||||
class DataTypeArray final : public IDataType
|
||||
{
|
||||
private:
|
||||
/// Тип элементов массивов.
|
||||
|
@ -10,7 +10,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeDate : public IDataTypeNumberFixed<UInt16, ColumnUInt16>
|
||||
class DataTypeDate final : public IDataTypeNumberFixed<UInt16, ColumnUInt16>
|
||||
{
|
||||
public:
|
||||
DataTypeDate() {}
|
||||
|
@ -10,7 +10,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeDateTime : public IDataTypeNumberFixed<UInt32, ColumnUInt32>
|
||||
class DataTypeDateTime final : public IDataTypeNumberFixed<UInt32, ColumnUInt32>
|
||||
{
|
||||
public:
|
||||
DataTypeDateTime() {}
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
/**
|
||||
* Лямбда-выражение.
|
||||
*/
|
||||
class DataTypeExpression : public IDataTypeDummy
|
||||
class DataTypeExpression final : public IDataTypeDummy
|
||||
{
|
||||
private:
|
||||
DataTypes argument_types;
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeFixedString : public IDataType
|
||||
class DataTypeFixedString final : public IDataType
|
||||
{
|
||||
private:
|
||||
size_t n;
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeNested : public IDataType
|
||||
class DataTypeNested final : public IDataType
|
||||
{
|
||||
private:
|
||||
/// Имена и типы вложенных массивов.
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
/** Тип данных, соответствующий множеству значений в секции IN.
|
||||
* Используется только как промежуточный вариант при вычислении выражений.
|
||||
*/
|
||||
class DataTypeSet : public IDataTypeDummy
|
||||
class DataTypeSet final : public IDataTypeDummy
|
||||
{
|
||||
public:
|
||||
std::string getName() const { return "Set"; }
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeString : public IDataType
|
||||
class DataTypeString final : public IDataType
|
||||
{
|
||||
public:
|
||||
using FieldType = String;
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
* Также может быть использовать в качестве столбца - результата выполнения запроса.
|
||||
* Не может быть сохранён в таблицы.
|
||||
*/
|
||||
class DataTypeTuple : public IDataType
|
||||
class DataTypeTuple final : public IDataType
|
||||
{
|
||||
private:
|
||||
DataTypes elems;
|
||||
@ -138,6 +138,7 @@ public:
|
||||
{
|
||||
ColumnWithNameAndType col;
|
||||
col.column = elems[i]->createColumn();
|
||||
col.type = elems[i]->clone();
|
||||
tuple_block.insert(col);
|
||||
}
|
||||
return new ColumnTuple(tuple_block);
|
||||
|
@ -13,11 +13,11 @@ template <typename T>
|
||||
struct DataTypeFromFieldType;
|
||||
|
||||
#define DEFINE_DATA_TYPE_NUMBER_FIXED(TYPE) \
|
||||
class DataType ## TYPE : public IDataTypeNumberFixed<TYPE, Column ## TYPE> \
|
||||
class DataType ## TYPE final : public IDataTypeNumberFixed<TYPE, Column ## TYPE> \
|
||||
{ \
|
||||
public: \
|
||||
std::string getName() const { return #TYPE; } \
|
||||
DataTypePtr clone() const { return new DataType ## TYPE; } \
|
||||
DataTypePtr clone() const { return new DataType ## TYPE; } \
|
||||
}; \
|
||||
\
|
||||
template <> struct DataTypeFromFieldType<TYPE> \
|
||||
|
@ -42,6 +42,8 @@ public:
|
||||
: CacheDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.size}
|
||||
{}
|
||||
|
||||
std::exception_ptr getCreationException() const override { return {}; }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
std::string getTypeName() const override { return "Cache"; }
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -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",
|
||||
|
@ -25,8 +25,17 @@ public:
|
||||
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime)
|
||||
{
|
||||
createAttributes();
|
||||
loadData();
|
||||
calculateBytesAllocated();
|
||||
|
||||
try
|
||||
{
|
||||
loadData();
|
||||
calculateBytesAllocated();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
creation_exception = std::current_exception();
|
||||
}
|
||||
|
||||
creation_time = std::chrono::system_clock::now();
|
||||
}
|
||||
|
||||
@ -34,6 +43,8 @@ public:
|
||||
: FlatDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime}
|
||||
{}
|
||||
|
||||
std::exception_ptr getCreationException() const override { return creation_exception; }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
std::string getTypeName() const override { return "Flat"; }
|
||||
@ -398,10 +409,11 @@ private:
|
||||
std::size_t bytes_allocated = 0;
|
||||
std::size_t element_count = 0;
|
||||
std::size_t bucket_count = 0;
|
||||
mutable std::atomic<std::size_t> query_count;
|
||||
|
||||
std::chrono::time_point<std::chrono::system_clock> creation_time;
|
||||
|
||||
mutable std::atomic<std::size_t> query_count;
|
||||
std::exception_ptr creation_exception;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -22,8 +22,17 @@ public:
|
||||
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime)
|
||||
{
|
||||
createAttributes();
|
||||
loadData();
|
||||
calculateBytesAllocated();
|
||||
|
||||
try
|
||||
{
|
||||
loadData();
|
||||
calculateBytesAllocated();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
creation_exception = std::current_exception();
|
||||
}
|
||||
|
||||
creation_time = std::chrono::system_clock::now();
|
||||
}
|
||||
|
||||
@ -31,6 +40,8 @@ public:
|
||||
: HashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime}
|
||||
{}
|
||||
|
||||
std::exception_ptr getCreationException() const override { return creation_exception; }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
std::string getTypeName() const override { return "Hashed"; }
|
||||
@ -389,6 +400,8 @@ private:
|
||||
mutable std::atomic<std::size_t> query_count{};
|
||||
|
||||
std::chrono::time_point<std::chrono::system_clock> creation_time;
|
||||
|
||||
std::exception_ptr creation_exception;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -24,6 +24,8 @@ class IDictionary
|
||||
public:
|
||||
using id_t = std::uint64_t;
|
||||
|
||||
virtual std::exception_ptr getCreationException() const = 0;
|
||||
|
||||
virtual std::string getName() const = 0;
|
||||
|
||||
virtual std::string getTypeName() const = 0;
|
||||
|
@ -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;
|
||||
|
@ -90,7 +90,7 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
String getName() const override { return "MySQLBlockInputStream"; }
|
||||
String getName() const override { return "MySQL"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
private:
|
||||
Block readImpl() override { return stream->read(); }
|
||||
|
||||
String getName() const override { return "OwningBufferBlockInputStream"; }
|
||||
String getName() const override { return "OwningBuffer"; }
|
||||
|
||||
String getID() const override { return "OwningBuffer(" + stream->getID() + ")"; }
|
||||
|
||||
|
@ -26,6 +26,10 @@ namespace DB
|
||||
* - даты;
|
||||
* - даты-с-временем;
|
||||
* внутри каждой группы, но не из разных групп.
|
||||
*
|
||||
* Исключение: можно сравнивать дату и дату-с-временем с константной строкой. Пример: EventDate = '2015-01-01'.
|
||||
*
|
||||
* TODO Массивы, кортежи.
|
||||
*/
|
||||
|
||||
/** Игнорируем warning о сравнении signed и unsigned.
|
||||
@ -391,9 +395,9 @@ public:
|
||||
|
||||
private:
|
||||
template <typename T0, typename T1>
|
||||
bool executeNumRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
||||
bool executeNumRightType(Block & block, size_t result, const ColumnVector<T0> * col_left, const IColumn * col_right_untyped)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = typeid_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
if (const ColumnVector<T1> * col_right = typeid_cast<const ColumnVector<T1> *>(col_right_untyped))
|
||||
{
|
||||
ColumnUInt8 * col_res = new ColumnUInt8;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -404,7 +408,7 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T1> * col_right = typeid_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
else if (const ColumnConst<T1> * col_right = typeid_cast<const ColumnConst<T1> *>(col_right_untyped))
|
||||
{
|
||||
ColumnUInt8 * col_res = new ColumnUInt8;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -420,9 +424,9 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeNumConstRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst<T0> * col_left)
|
||||
bool executeNumConstRightType(Block & block, size_t result, const ColumnConst<T0> * col_left, const IColumn * col_right_untyped)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = typeid_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
if (const ColumnVector<T1> * col_right = typeid_cast<const ColumnVector<T1> *>(col_right_untyped))
|
||||
{
|
||||
ColumnUInt8 * col_res = new ColumnUInt8;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -433,7 +437,7 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T1> * col_right = typeid_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
else if (const ColumnConst<T1> * col_right = typeid_cast<const ColumnConst<T1> *>(col_right_untyped))
|
||||
{
|
||||
UInt8 res = 0;
|
||||
NumComparisonImpl<T0, T1, Op<T0, T1>>::constant_constant(col_left->getData(), col_right->getData(), res);
|
||||
@ -448,41 +452,41 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeNumLeftType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
bool executeNumLeftType(Block & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped)
|
||||
{
|
||||
if (ColumnVector<T0> * col_left = typeid_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
if (const ColumnVector<T0> * col_left = typeid_cast<const ColumnVector<T0> *>(col_left_untyped))
|
||||
{
|
||||
if ( executeNumRightType<T0, UInt8>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, UInt16>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, UInt32>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, UInt64>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Int8>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Int16>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Int32>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Int64>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Float32>(block, arguments, result, col_left)
|
||||
|| executeNumRightType<T0, Float64>(block, arguments, result, col_left))
|
||||
if ( executeNumRightType<T0, UInt8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Float32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Float64>(block, result, col_left, col_right_untyped))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
throw Exception("Illegal column " + col_right_untyped->getName()
|
||||
+ " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (ColumnConst<T0> * col_left = typeid_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
else if (const ColumnConst<T0> * col_left = typeid_cast<const ColumnConst<T0> *>(col_left_untyped))
|
||||
{
|
||||
if ( executeNumConstRightType<T0, UInt8>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, UInt16>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, UInt32>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, UInt64>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Int8>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Int16>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Int32>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Int64>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Float32>(block, arguments, result, col_left)
|
||||
|| executeNumConstRightType<T0, Float64>(block, arguments, result, col_left))
|
||||
if ( executeNumConstRightType<T0, UInt8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Float32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Float64>(block, result, col_left, col_right_untyped))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
throw Exception("Illegal column " + col_right_untyped->getName()
|
||||
+ " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -490,17 +494,14 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
void executeString(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
void executeString(Block & block, size_t result, const IColumn * c0, const IColumn * c1)
|
||||
{
|
||||
IColumn * c0 = &*block.getByPosition(arguments[0]).column;
|
||||
IColumn * c1 = &*block.getByPosition(arguments[1]).column;
|
||||
|
||||
ColumnString * c0_string = typeid_cast<ColumnString *>(c0);
|
||||
ColumnString * c1_string = typeid_cast<ColumnString *>(c1);
|
||||
ColumnFixedString * c0_fixed_string = typeid_cast<ColumnFixedString *>(c0);
|
||||
ColumnFixedString * c1_fixed_string = typeid_cast<ColumnFixedString *>(c1);
|
||||
ColumnConstString * c0_const = typeid_cast<ColumnConstString *>(c0);
|
||||
ColumnConstString * c1_const = typeid_cast<ColumnConstString *>(c1);
|
||||
const ColumnString * c0_string = typeid_cast<const ColumnString *>(c0);
|
||||
const ColumnString * c1_string = typeid_cast<const ColumnString *>(c1);
|
||||
const ColumnFixedString * c0_fixed_string = typeid_cast<const ColumnFixedString *>(c0);
|
||||
const ColumnFixedString * c1_fixed_string = typeid_cast<const ColumnFixedString *>(c1);
|
||||
const ColumnConstString * c0_const = typeid_cast<const ColumnConstString *>(c0);
|
||||
const ColumnConstString * c1_const = typeid_cast<const ColumnConstString *>(c1);
|
||||
|
||||
using StringImpl = StringComparisonImpl<Op<int, int>>;
|
||||
|
||||
@ -559,13 +560,66 @@ private:
|
||||
c_res->getData());
|
||||
else
|
||||
throw Exception("Illegal columns "
|
||||
+ block.getByPosition(arguments[0]).column->getName() + " and "
|
||||
+ block.getByPosition(arguments[1]).column->getName()
|
||||
+ c0->getName() + " and " + c1->getName()
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
void executeDateOrDateTimeWithConstString(Block & block, size_t result,
|
||||
const IColumn * col_left_untyped, const IColumn * col_right_untyped,
|
||||
bool left_is_num, bool right_is_num)
|
||||
{
|
||||
/// Особый случай - сравнение дат и дат-с-временем со строковой константой.
|
||||
const IColumn * column_date_or_datetime = left_is_num ? col_left_untyped : col_right_untyped;
|
||||
const IColumn * column_string_untyped = !left_is_num ? col_left_untyped : col_right_untyped;
|
||||
|
||||
bool is_date = false;
|
||||
bool is_date_time = false;
|
||||
|
||||
is_date = typeid_cast<const ColumnVector<DataTypeDate::FieldType> *>(column_date_or_datetime)
|
||||
|| typeid_cast<const ColumnConst<DataTypeDate::FieldType> *>(column_date_or_datetime);
|
||||
|
||||
if (!is_date)
|
||||
is_date_time = typeid_cast<const ColumnVector<DataTypeDateTime::FieldType> *>(column_date_or_datetime)
|
||||
|| typeid_cast<const ColumnConst<DataTypeDateTime::FieldType> *>(column_date_or_datetime);
|
||||
|
||||
const ColumnConstString * column_string = typeid_cast<const ColumnConstString *>(column_string_untyped);
|
||||
|
||||
if (!column_string
|
||||
|| (!is_date && !is_date_time))
|
||||
throw Exception("Illegal columns " + col_left_untyped->getName() + " and " + col_right_untyped->getName()
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (is_date)
|
||||
{
|
||||
DayNum_t date;
|
||||
ReadBufferFromString in(column_string->getData());
|
||||
readDateText(date, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for Date: " + column_string->getData());
|
||||
|
||||
ColumnConst<DataTypeDate::FieldType> parsed_const_date(block.rowsInFirstColumn(), date);
|
||||
executeNumLeftType<DataTypeDate::FieldType>(block, result,
|
||||
left_is_num ? col_left_untyped : &parsed_const_date,
|
||||
left_is_num ? &parsed_const_date : col_right_untyped);
|
||||
}
|
||||
else if (is_date_time)
|
||||
{
|
||||
time_t date_time;
|
||||
ReadBufferFromString in(column_string->getData());
|
||||
readDateTimeText(date_time, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for DateTime: " + column_string->getData());
|
||||
|
||||
ColumnConst<DataTypeDateTime::FieldType> parsed_const_date_time(block.rowsInFirstColumn(), date_time);
|
||||
executeNumLeftType<DataTypeDateTime::FieldType>(block, result,
|
||||
left_is_num ? col_left_untyped : &parsed_const_date_time,
|
||||
left_is_num ? &parsed_const_date_time : col_right_untyped);
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
@ -581,12 +635,36 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!( ( arguments[0]->isNumeric() && arguments[0]->behavesAsNumber()
|
||||
&& arguments[1]->isNumeric() && arguments[1]->behavesAsNumber())
|
||||
|| ( (arguments[0]->getName() == "String" || arguments[0]->getName().substr(0, 11) == "FixedString")
|
||||
&& (arguments[1]->getName() == "String" || arguments[1]->getName().substr(0, 11) == "FixedString"))
|
||||
|| (arguments[0]->getName() == "Date" && arguments[1]->getName() == "Date")
|
||||
|| (arguments[0]->getName() == "DateTime" && arguments[1]->getName() == "DateTime")))
|
||||
bool left_is_date = false;
|
||||
bool left_is_date_time = false;
|
||||
bool left_is_string = false;
|
||||
bool left_is_fixed_string = false;
|
||||
|
||||
false
|
||||
|| (left_is_date = typeid_cast<const DataTypeDate *>(arguments[0].get()))
|
||||
|| (left_is_date_time = typeid_cast<const DataTypeDateTime *>(arguments[0].get()))
|
||||
|| (left_is_string = typeid_cast<const DataTypeString *>(arguments[0].get()))
|
||||
|| (left_is_fixed_string = typeid_cast<const DataTypeFixedString *>(arguments[0].get()));
|
||||
|
||||
bool right_is_date = false;
|
||||
bool right_is_date_time = false;
|
||||
bool right_is_string = false;
|
||||
bool right_is_fixed_string = false;
|
||||
|
||||
false
|
||||
|| (right_is_date = typeid_cast<const DataTypeDate *>(arguments[1].get()))
|
||||
|| (right_is_date_time = typeid_cast<const DataTypeDateTime *>(arguments[1].get()))
|
||||
|| (right_is_string = typeid_cast<const DataTypeString *>(arguments[1].get()))
|
||||
|| (right_is_fixed_string = typeid_cast<const DataTypeFixedString *>(arguments[1].get()));
|
||||
|
||||
if (!( (arguments[0]->behavesAsNumber() && arguments[1]->behavesAsNumber())
|
||||
|| ((left_is_string || left_is_fixed_string) && (right_is_string || right_is_fixed_string))
|
||||
|| (left_is_date && right_is_date)
|
||||
|| (left_is_date && right_is_string) /// Можно сравнивать дату и дату-с-временем с константной строкой.
|
||||
|| (left_is_string && right_is_date)
|
||||
|| (left_is_date_time && right_is_date_time)
|
||||
|| (left_is_date_time && right_is_string)
|
||||
|| (left_is_string && right_is_date_time)))
|
||||
throw Exception("Illegal types of arguments (" + arguments[0]->getName() + ", " + arguments[1]->getName() + ")"
|
||||
" of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -596,24 +674,36 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (block.getByPosition(arguments[0]).column->isNumeric())
|
||||
const IColumn * col_left_untyped = block.getByPosition(arguments[0]).column.get();
|
||||
const IColumn * col_right_untyped = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
bool left_is_num = col_left_untyped->isNumeric();
|
||||
bool right_is_num = col_right_untyped->isNumeric();
|
||||
|
||||
if (left_is_num && right_is_num)
|
||||
{
|
||||
if (!( executeNumLeftType<UInt8>(block, arguments, result)
|
||||
|| executeNumLeftType<UInt16>(block, arguments, result)
|
||||
|| executeNumLeftType<UInt32>(block, arguments, result)
|
||||
|| executeNumLeftType<UInt64>(block, arguments, result)
|
||||
|| executeNumLeftType<Int8>(block, arguments, result)
|
||||
|| executeNumLeftType<Int16>(block, arguments, result)
|
||||
|| executeNumLeftType<Int32>(block, arguments, result)
|
||||
|| executeNumLeftType<Int64>(block, arguments, result)
|
||||
|| executeNumLeftType<Float32>(block, arguments, result)
|
||||
|| executeNumLeftType<Float64>(block, arguments, result)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
if (!( executeNumLeftType<UInt8>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt16>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt32>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt64>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int8>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int16>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int32>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int64>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Float32>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Float64>(block, result, col_left_untyped, col_right_untyped)))
|
||||
throw Exception("Illegal column " + col_left_untyped->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (!left_is_num && !right_is_num)
|
||||
{
|
||||
executeString(block, result, col_left_untyped, col_right_untyped);
|
||||
}
|
||||
else
|
||||
executeString(block, arguments, result);
|
||||
{
|
||||
executeDateOrDateTimeWithConstString(block, result, col_left_untyped, col_right_untyped, left_is_num, right_is_num);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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>
|
||||
@ -13,6 +12,7 @@
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/DataTypes/DataTypeTuple.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
#include <DB/DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
@ -21,12 +21,13 @@
|
||||
#include <DB/Columns/ColumnTuple.h>
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnReplicated.h>
|
||||
#include <DB/Columns/ColumnAggregateFunction.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 +59,12 @@ namespace DB
|
||||
*
|
||||
* bar(x, min, max, width) - рисует полосу из количества символов, пропорционального (x - min) и равного width при x == max.
|
||||
*
|
||||
* transform(x, from_array, to_array[, default]) - преобразовать x согласно переданному явным образом соответствию.
|
||||
* version() - возвращает текущую версию сервера в строке.
|
||||
*
|
||||
* finalizeAggregation(agg_state) - по состоянию агрегации получить результат.
|
||||
*
|
||||
* runningAccumulate(agg_state) - принимает состояния агрегатной функции и возвращает столбец со значениями,
|
||||
* являющимися результатом накопления этих состояний для множества строк блока, от первой до текущей строки.
|
||||
*/
|
||||
|
||||
|
||||
@ -112,6 +118,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Получить имя хоста. (Оно - константа, вычисляется один раз за весь запрос.)
|
||||
class FunctionHostName : public IFunction
|
||||
{
|
||||
@ -146,6 +153,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionVisibleWidth : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -890,504 +898,127 @@ 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 и вычислим функцию как обычно.
|
||||
std::ostringstream os;
|
||||
os << DBMS_VERSION_MAJOR << "." << DBMS_VERSION_MINOR << "." << Revision::get();
|
||||
return os.str();
|
||||
}
|
||||
};
|
||||
|
||||
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);
|
||||
/** Весьма необычная функция.
|
||||
* Принимает состояние агрегатной функции (например runningAccumulate(uniqState(UserID))),
|
||||
* и для каждой строки блока, возвращает результат агрегатной функции по объединению состояний от всех предыдущих строк блока и текущей строки.
|
||||
*
|
||||
* То есть, функция зависит от разбиения данных на блоки и от порядка строк в блоке.
|
||||
*/
|
||||
class FunctionRunningAccumulate : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "runningAccumulate";
|
||||
static IFunction * create(const Context & context) { return new FunctionRunningAccumulate; }
|
||||
|
||||
for (size_t i = 1; i < arguments.size(); ++i)
|
||||
{
|
||||
tmp_block.insert(block.getByPosition(arguments[i]));
|
||||
tmp_arguments.push_back(i);
|
||||
}
|
||||
String getName() const override { return name; }
|
||||
|
||||
tmp_block.insert(block.getByPosition(result));
|
||||
size_t tmp_result = arguments.size();
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
execute(tmp_block, tmp_arguments, tmp_result);
|
||||
const DataTypeAggregateFunction * type = typeid_cast<const DataTypeAggregateFunction *>(&*arguments[0]);
|
||||
if (!type)
|
||||
throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(
|
||||
block.rowsInFirstColumn(),
|
||||
(*tmp_block.getByPosition(tmp_result).column)[0]);
|
||||
return type->getReturnType()->clone();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNum(const IColumn * in_untyped, IColumn * out_untyped)
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
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))
|
||||
const ColumnAggregateFunction * column_with_states = typeid_cast<const ColumnAggregateFunction *>(&*block.getByPosition(arguments.at(0)).column);
|
||||
if (!column_with_states)
|
||||
throw Exception(
|
||||
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
|
||||
"Illegal column " + block.getByPosition(arguments.at(0)).column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return true;
|
||||
}
|
||||
AggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction();
|
||||
const IAggregateFunction & agg_func = *aggregate_function_ptr;
|
||||
|
||||
return false;
|
||||
}
|
||||
auto deleter = [&agg_func] (char * ptr) { agg_func.destroy(ptr); free(ptr); };
|
||||
std::unique_ptr<char, decltype(deleter)> place { reinterpret_cast<char *>(malloc(agg_func.sizeOfData())), deleter };
|
||||
|
||||
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;
|
||||
agg_func.create(place.get()); /// Немного не exception-safe. Если здесь выкинется исключение, то зря вызовется destroy.
|
||||
|
||||
executeImplNumToNumWithDefault<T, U>(in->getData(), out->getData(), default_value.get<U>());
|
||||
return true;
|
||||
}
|
||||
ColumnPtr result_column_ptr = agg_func.getReturnType()->createColumn();
|
||||
block.getByPosition(result).column = result_column_ptr;
|
||||
IColumn & result_column = *result_column_ptr;
|
||||
result_column.reserve(column_with_states->size());
|
||||
|
||||
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 auto & states = column_with_states->getData();
|
||||
for (const auto & state_to_add : states)
|
||||
{
|
||||
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;
|
||||
agg_func.merge(place.get(), state_to_add);
|
||||
agg_func.insertResultInto(place.get(), result_column);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
void executeImplNumToNum(const PODArray<T> & src, PODArray<T> & dst)
|
||||
|
||||
/** Принимает состояние агрегатной функции. Возвращает результат агрегации.
|
||||
*/
|
||||
class FunctionFinalizeAggregation : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "finalizeAggregation";
|
||||
static IFunction * create(const Context & context) { return new FunctionFinalizeAggregation; }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
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];
|
||||
}
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeAggregateFunction * type = typeid_cast<const DataTypeAggregateFunction *>(&*arguments[0]);
|
||||
if (!type)
|
||||
throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return type->getReturnType()->clone();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void executeImplNumToStringWithDefault(const PODArray<T> & src,
|
||||
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
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;
|
||||
}
|
||||
}
|
||||
ColumnAggregateFunction * column_with_states = typeid_cast<ColumnAggregateFunction *>(&*block.getByPosition(arguments.at(0)).column);
|
||||
if (!column_with_states)
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments.at(0)).column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
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;
|
||||
block.getByPosition(result).column = column_with_states->convertToValues();
|
||||
}
|
||||
};
|
||||
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -14,6 +14,9 @@
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
|
||||
#include <emmintrin.h>
|
||||
#include <nmmintrin.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -199,9 +202,7 @@ struct LengthUTF8Impl
|
||||
};
|
||||
|
||||
|
||||
/** Переводит строку в нижний (верхний) регистр, в текущей локали, в однобайтовой кодировке.
|
||||
*/
|
||||
template <int F(int)>
|
||||
template <char not_case_lower_bound, char not_case_upper_bound>
|
||||
struct LowerUpperImpl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,
|
||||
@ -209,81 +210,249 @@ struct LowerUpperImpl
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
res_offsets.assign(offsets);
|
||||
array(&*data.begin(), &*data.end(), &*res_data.begin());
|
||||
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)
|
||||
ColumnString::Chars_t & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
array(&*data.begin(), &*data.end(), &*res_data.begin());
|
||||
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.begin()), reinterpret_cast<const UInt8 *>(&*data.end()),
|
||||
reinterpret_cast<UInt8 *>(&*res_data.begin()));
|
||||
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)
|
||||
*dst = F(*src);
|
||||
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);
|
||||
}
|
||||
};
|
||||
|
||||
/** Если строка содержит текст в кодировке UTF-8 - перевести его в нижний (верхний) регистр.
|
||||
* Замечание: предполагается, что после перевода символа в другой регистр,
|
||||
* длина его мультибайтовой последовательности в UTF-8 не меняется.
|
||||
* Иначе - поведение не определено.
|
||||
*/
|
||||
template <int F(int)>
|
||||
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 LowerUpperUTF8Impl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,
|
||||
ColumnString::Chars_t & res_data, ColumnString::Offsets_t & res_offsets)
|
||||
ColumnString::Chars_t & res_data, ColumnString::Offsets_t & res_offsets)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
res_offsets.assign(offsets);
|
||||
array(&*data.begin(), &*data.end(), &*res_data.begin());
|
||||
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)
|
||||
ColumnString::Chars_t & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
array(&*data.begin(), &*data.end(), &*res_data.begin());
|
||||
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.begin()), reinterpret_cast<const UInt8 *>(&*data.end()),
|
||||
reinterpret_cast<UInt8 *>(&*res_data.begin()));
|
||||
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 Poco::UTF8Encoding utf8;
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
|
||||
while (src < src_end)
|
||||
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)
|
||||
{
|
||||
int chars = utf8.convert(F(utf8.convert(src)), dst, src_end - src);
|
||||
if (chars)
|
||||
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)
|
||||
{
|
||||
src += chars;
|
||||
dst += chars;
|
||||
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
|
||||
{
|
||||
++src;
|
||||
++dst;
|
||||
/// 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;
|
||||
}
|
||||
};
|
||||
|
||||
@ -1428,10 +1597,14 @@ typedef FunctionStringOrArrayToT<EmptyImpl<false>, NameEmpty, UInt8> Function
|
||||
typedef FunctionStringOrArrayToT<EmptyImpl<true>, NameNotEmpty, UInt8> FunctionNotEmpty;
|
||||
typedef FunctionStringOrArrayToT<LengthImpl, NameLength, UInt64> FunctionLength;
|
||||
typedef FunctionStringOrArrayToT<LengthUTF8Impl, NameLengthUTF8, UInt64> FunctionLengthUTF8;
|
||||
typedef FunctionStringToString<LowerUpperImpl<tolower>, NameLower> FunctionLower;
|
||||
typedef FunctionStringToString<LowerUpperImpl<toupper>, NameUpper> FunctionUpper;
|
||||
typedef FunctionStringToString<LowerUpperUTF8Impl<Poco::Unicode::toLower>, NameLowerUTF8> FunctionLowerUTF8;
|
||||
typedef FunctionStringToString<LowerUpperUTF8Impl<Poco::Unicode::toUpper>, NameUpperUTF8> FunctionUpperUTF8;
|
||||
typedef FunctionStringToString<LowerUpperImpl<'A', 'Z'>, NameLower> FunctionLower;
|
||||
typedef FunctionStringToString<LowerUpperImpl<'a', 'z'>, NameUpper> FunctionUpper;
|
||||
typedef FunctionStringToString<
|
||||
LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>,
|
||||
NameLowerUTF8> FunctionLowerUTF8;
|
||||
typedef FunctionStringToString<
|
||||
LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>,
|
||||
NameUpperUTF8> FunctionUpperUTF8;
|
||||
typedef FunctionStringToString<ReverseImpl, NameReverse> FunctionReverse;
|
||||
typedef FunctionStringToString<ReverseUTF8Impl, NameReverseUTF8> FunctionReverseUTF8;
|
||||
typedef FunctionStringNumNumToString<SubstringImpl, NameSubstring> FunctionSubstring;
|
||||
|
810
dbms/include/DB/Functions/FunctionsTransform.h
Normal file
810
dbms/include/DB/Functions/FunctionsTransform.h
Normal 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;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -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
|
||||
};
|
||||
|
||||
/** Формат сжатого блока следующий:
|
||||
|
@ -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");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -22,6 +22,7 @@ struct SubqueryForSet
|
||||
{
|
||||
/// Источник - получен с помощью InterpreterSelectQuery подзапроса.
|
||||
BlockInputStreamPtr source;
|
||||
Block source_sample;
|
||||
|
||||
/// Если задано - создать из результата Set.
|
||||
SetPtr set;
|
||||
|
@ -1,23 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Dictionaries/IDictionary.h>
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
#include <Yandex/MultiVersion.h>
|
||||
#include <Yandex/logger_useful.h>
|
||||
#include <Poco/Event.h>
|
||||
#include <unistd.h>
|
||||
#include <time.h>
|
||||
#include <mutex>
|
||||
#include <thread>
|
||||
#include <unordered_map>
|
||||
#include <chrono>
|
||||
#include <random>
|
||||
#include <unistd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class IDictionary;
|
||||
|
||||
/** Manages user-defined dictionaries.
|
||||
* Monitors configuration file and automatically reloads dictionaries in a separate thread.
|
||||
@ -43,11 +43,23 @@ 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;
|
||||
};
|
||||
|
||||
struct failed_dictionary_info final
|
||||
{
|
||||
std::unique_ptr<IDictionary> dict;
|
||||
std::chrono::system_clock::time_point next_attempt_time;
|
||||
std::uint64_t error_count;
|
||||
};
|
||||
|
||||
std::unordered_map<std::string, dictionary_info> dictionaries;
|
||||
std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
|
||||
std::unordered_map<std::string, failed_dictionary_info> failed_dictionaries;
|
||||
std::mt19937_64 rnd_engine{getSeed()};
|
||||
|
||||
Context & context;
|
||||
@ -77,7 +89,7 @@ private:
|
||||
{
|
||||
timespec ts;
|
||||
clock_gettime(CLOCK_MONOTONIC, &ts);
|
||||
return ts.tv_nsec ^ getpid();
|
||||
return static_cast<std::uint64_t>(ts.tv_nsec ^ getpid());
|
||||
}
|
||||
|
||||
public:
|
||||
@ -95,24 +107,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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
|
@ -130,6 +130,15 @@ private:
|
||||
|
||||
void ignoreWithTotals();
|
||||
|
||||
/** Если в запросе SELECT есть секция SETTINGS, то применить настройки из неё.
|
||||
* Затем достать настройки из context и поместить их в settings.
|
||||
*
|
||||
* Секция SETTINGS - настройки для конкретного запроса.
|
||||
* Обычно настройки могут быть переданы другими способами, не внутри запроса.
|
||||
* Но использование такой секции оправдано, если нужно задать настройки для одного подзапроса.
|
||||
*/
|
||||
void initSettings();
|
||||
|
||||
ASTPtr query_ptr;
|
||||
ASTSelectQuery & query;
|
||||
Context context;
|
||||
|
@ -8,7 +8,7 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Установить один или несколько параметров, для сессии или глобально.
|
||||
/** Установить один или несколько параметров, для сессии или глобально... или для текущего запроса.
|
||||
*/
|
||||
class InterpreterSetQuery
|
||||
{
|
||||
@ -16,17 +16,36 @@ public:
|
||||
InterpreterSetQuery(ASTPtr query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_) {}
|
||||
|
||||
|
||||
/** Обычный запрос SET. Задать настройку на сессию или глобальную (если указано GLOBAL).
|
||||
*/
|
||||
void execute()
|
||||
{
|
||||
ASTSetQuery & ast = typeid_cast<ASTSetQuery &>(*query_ptr);
|
||||
|
||||
Context & target = ast.global ? context.getGlobalContext() : context.getSessionContext();
|
||||
executeImpl(ast, target);
|
||||
}
|
||||
|
||||
/** Задать настроку для текущего контекста (контекста запроса).
|
||||
* Используется для интерпретации секции SETTINGS в запросе SELECT.
|
||||
*/
|
||||
void executeForCurrentContext()
|
||||
{
|
||||
ASTSetQuery & ast = typeid_cast<ASTSetQuery &>(*query_ptr);
|
||||
executeImpl(ast, context);
|
||||
}
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context & context;
|
||||
|
||||
void executeImpl(ASTSetQuery & ast, Context & target)
|
||||
{
|
||||
/** Значение readonly понимается следующим образом:
|
||||
* 0 - можно всё.
|
||||
* 1 - можно делать только запросы на чтение; в том числе, нельзя менять настройки.
|
||||
* 2 - можно делать только запросы на чтение и можно менять настройки, кроме настройки readonly.
|
||||
*/
|
||||
* 0 - можно всё.
|
||||
* 1 - можно делать только запросы на чтение; в том числе, нельзя менять настройки.
|
||||
* 2 - можно делать только запросы на чтение и можно менять настройки, кроме настройки readonly.
|
||||
*/
|
||||
|
||||
if (context.getSettingsRef().limits.readonly == 1)
|
||||
throw Exception("Cannot execute SET query in readonly mode", ErrorCodes::READONLY);
|
||||
@ -39,10 +58,6 @@ public:
|
||||
for (ASTSetQuery::Changes::const_iterator it = ast.changes.begin(); it != ast.changes.end(); ++it)
|
||||
target.setSetting(it->name, it->value);
|
||||
}
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context & context;
|
||||
};
|
||||
|
||||
|
||||
|
@ -74,6 +74,11 @@ public:
|
||||
|
||||
bool empty() { return type == Type::EMPTY; }
|
||||
|
||||
/** Передать информацию о структуре блока.
|
||||
* Следует обязательно вызвать до вызовов insertFromBlock.
|
||||
*/
|
||||
void setSampleBlock(const Block & block);
|
||||
|
||||
/** Добавить в отображение для соединения блок "правой" таблицы.
|
||||
* Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
|
||||
*/
|
||||
@ -217,6 +222,9 @@ private:
|
||||
bool keys_fit_128_bits;
|
||||
Sizes key_sizes;
|
||||
|
||||
Block sample_block_with_columns_to_add;
|
||||
Block sample_block_with_keys;
|
||||
|
||||
Logger * log;
|
||||
|
||||
/// Ограничения на максимальный размер множества
|
||||
@ -243,6 +251,9 @@ private:
|
||||
|
||||
/// Проверить не превышены ли допустимые размеры множества
|
||||
bool checkSizeLimits() const;
|
||||
|
||||
/// Кинуть исключение, если в блоках не совпадают типы ключей.
|
||||
void checkTypesOfKeys(const Block & block_left, const Block & block_right) const;
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<Join> JoinPtr;
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
|
@ -283,7 +283,7 @@ public:
|
||||
* node - это список значений: 1, 2, 3 или список tuple-ов: (1, 2), (3, 4), (5, 6).
|
||||
* create_ordered_set - создавать ли вектор упорядоченных элементов. Нужен для работы индекса
|
||||
*/
|
||||
void createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set);
|
||||
void createFromAST(DataTypes & types, ASTPtr node, const Context & context, bool create_ordered_set);
|
||||
|
||||
// Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
|
||||
bool insertFromBlock(const Block & block, bool create_ordered_set = false);
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -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;
|
||||
@ -31,153 +51,8 @@ public:
|
||||
ASTPtr order_expression_list;
|
||||
ASTPtr limit_offset;
|
||||
ASTPtr limit_length;
|
||||
ASTPtr settings;
|
||||
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;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -12,9 +12,15 @@ namespace DB
|
||||
*/
|
||||
class ParserSetQuery : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserSetQuery(bool parse_only_internals_ = false) : parse_only_internals(parse_only_internals_) {}
|
||||
|
||||
protected:
|
||||
const char * getName() const { return "SET query"; }
|
||||
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
|
||||
|
||||
/// Парсить список name = value пар, без SET [GLOBAL].
|
||||
bool parse_only_internals;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
|
||||
|
@ -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};
|
||||
};
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/Common/formatReadable.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -116,8 +117,8 @@ public:
|
||||
{
|
||||
size_t free_bytes = getUnreservedFreeSpace(path);
|
||||
if (free_bytes < size)
|
||||
throw Exception("Not enough free disk space to reserve: " + toString(free_bytes) + " available, "
|
||||
+ toString(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE);
|
||||
throw Exception("Not enough free disk space to reserve: " + formatReadableSizeWithBinarySuffix(free_bytes) + " available, "
|
||||
+ formatReadableSizeWithBinarySuffix(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE);
|
||||
return new Reservation(size);
|
||||
}
|
||||
|
||||
|
@ -43,9 +43,7 @@ public:
|
||||
{
|
||||
pre_column_names = prewhere_actions->getRequiredColumns();
|
||||
|
||||
/// @todo somehow decide which injected columns belong to PREWHERE, optimizing reads
|
||||
pre_column_names.insert(std::end(pre_column_names),
|
||||
std::begin(injected_columns), std::end(injected_columns));
|
||||
injectRequiredColumns(pre_column_names);
|
||||
|
||||
if (pre_column_names.empty())
|
||||
pre_column_names.push_back(column_names[0]);
|
||||
@ -100,7 +98,7 @@ public:
|
||||
setTotalRowsApprox(total_rows);
|
||||
}
|
||||
|
||||
String getName() const override { return "MergeTreeBlockInputStream"; }
|
||||
String getName() const override { return "MergeTree"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -15,7 +15,17 @@ public:
|
||||
|
||||
void writePrefix() override
|
||||
{
|
||||
storage.data.delayInsertIfNeeded();
|
||||
/// Если слишком много кусков - делаем внеочередные мерджи, синхронно, в текущем потоке.
|
||||
/// Почему 10? - на всякий случай, вместо бесконечного цикла.
|
||||
for (size_t i = 0; i < 10; ++i)
|
||||
{
|
||||
size_t parts_count = storage.data.getMaxPartsCountForMonth();
|
||||
if (parts_count <= storage.data.settings.parts_to_delay_insert)
|
||||
break;
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::SynchronousMergeOnInsert);
|
||||
storage.merge(0, true);
|
||||
}
|
||||
}
|
||||
|
||||
void write(const Block & block) override
|
||||
@ -26,6 +36,8 @@ public:
|
||||
UInt64 temp_index = storage.increment.get();
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, temp_index);
|
||||
storage.data.renameTempPartAndAdd(part, &storage.increment);
|
||||
|
||||
/// Инициируем асинхронный мердж - он будет произведён, если пора делать мердж и если в background_pool-е есть место.
|
||||
storage.merge_task_handle->wake();
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
||||
|
@ -49,7 +49,7 @@ public:
|
||||
DiskSpaceMonitor::Reservation * disk_reservation = nullptr);
|
||||
|
||||
/// Примерное количество места на диске, нужное для мерджа. С запасом.
|
||||
size_t estimateDiskSpaceForMerge(const MergeTreeData::DataPartsVector & parts);
|
||||
static size_t estimateDiskSpaceForMerge(const MergeTreeData::DataPartsVector & parts);
|
||||
|
||||
/** Отменяет все мерджи. Все выполняющиеся сейчас вызовы mergeParts скоро бросят исключение.
|
||||
* Все новые вызовы будут бросать исключения, пока не будет вызван uncancelAll().
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,12 +9,14 @@
|
||||
#include <DB/Parsers/ASTSubquery.h>
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
#include <DB/Common/escapeForFileName.h>
|
||||
#include <statdaemons/ext/scope_guard.hpp>
|
||||
#include <memory>
|
||||
#include <unordered_map>
|
||||
#include <map>
|
||||
#include <limits>
|
||||
#include <cstddef>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -34,6 +36,7 @@ class MergeTreeWhereOptimizer
|
||||
static constexpr auto max_columns_relative_size = 0.25f;
|
||||
static constexpr auto and_function_name = "and";
|
||||
static constexpr auto equals_function_name = "equals";
|
||||
static constexpr auto array_join_function_name = "arrayJoin";
|
||||
|
||||
public:
|
||||
MergeTreeWhereOptimizer(const MergeTreeWhereOptimizer&) = delete;
|
||||
@ -46,6 +49,7 @@ public:
|
||||
table_columns{toUnorderedSet(data.getColumnsList())}, log{log}
|
||||
{
|
||||
calculateColumnSizes(data, column_names);
|
||||
determineArrayJoinedNames(select);
|
||||
optimize(select);
|
||||
}
|
||||
|
||||
@ -96,28 +100,33 @@ private:
|
||||
{
|
||||
const auto condition = conditions[idx].get();
|
||||
|
||||
/// linearize sub-conjunctions
|
||||
if (const auto function = typeid_cast<ASTFunction *>(condition))
|
||||
{
|
||||
if (function->name == and_function_name)
|
||||
{
|
||||
for (auto & child : function->arguments->children)
|
||||
conditions.emplace_back(std::move(child));
|
||||
|
||||
/// remove the condition corresponding to conjunction
|
||||
remove_condition_at_index(idx);
|
||||
|
||||
/// continue iterating without increment to ensure the just added conditions are processed
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
SCOPE_EXIT(++idx);
|
||||
|
||||
if (cannotBeMoved(condition))
|
||||
continue;
|
||||
|
||||
IdentifierNameSet identifiers{};
|
||||
collectIdentifiersNoSubqueries(condition, identifiers);
|
||||
|
||||
/// do not take into consideration the conditions consisting only of primary key columns
|
||||
if (hasNonPrimaryKeyColumns(identifiers) && isSubsetOfTableColumns(identifiers))
|
||||
{
|
||||
/// linearize sub-conjunctions
|
||||
if (const auto function = typeid_cast<ASTFunction *>(condition))
|
||||
{
|
||||
if (function->name == and_function_name)
|
||||
{
|
||||
for (auto & child : function->arguments->children)
|
||||
conditions.emplace_back(std::move(child));
|
||||
|
||||
/// remove the condition corresponding to conjunction
|
||||
remove_condition_at_index(idx);
|
||||
|
||||
/// continue iterating without increment to ensure the just added conditions are processed
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
/// calculate size of columns involved in condition
|
||||
const auto cond_columns_size = getIdentifiersColumnSize(identifiers);
|
||||
|
||||
@ -129,8 +138,6 @@ private:
|
||||
good_or_viable_condition.second = cond_columns_size;
|
||||
}
|
||||
}
|
||||
|
||||
++idx;
|
||||
}
|
||||
|
||||
const auto move_condition_to_prewhere = [&] (const std::size_t idx) {
|
||||
@ -180,6 +187,10 @@ private:
|
||||
{
|
||||
auto & condition = select.where_expression;
|
||||
|
||||
/// do not optimize restricted expressions
|
||||
if (cannotBeMoved(select.where_expression.get()))
|
||||
return;
|
||||
|
||||
IdentifierNameSet identifiers{};
|
||||
collectIdentifiersNoSubqueries(condition, identifiers);
|
||||
|
||||
@ -300,11 +311,49 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
/** ARRAY JOIN'ed columns as well as arrayJoin() result cannot be used in PREWHERE, therefore expressions
|
||||
* containing said columns should not be moved to PREWHERE at all.
|
||||
* We assume all AS aliases have been expanded prior to using this class */
|
||||
bool cannotBeMoved(const IAST * ptr) const
|
||||
{
|
||||
if (const auto function_ptr = typeid_cast<const ASTFunction *>(ptr))
|
||||
{
|
||||
/// disallow arrayJoin expressions to be moved to PREWHERE for now
|
||||
if (array_join_function_name == function_ptr->name)
|
||||
return true;
|
||||
}
|
||||
else if (const auto identifier_ptr = typeid_cast<const ASTIdentifier *>(ptr))
|
||||
{
|
||||
/// disallow moving result of ARRAY JOIN to PREWHERE
|
||||
if (identifier_ptr->kind == ASTIdentifier::Column)
|
||||
if (array_joined_names.count(identifier_ptr->name) ||
|
||||
array_joined_names.count(DataTypeNested::extractNestedTableName(identifier_ptr->name)))
|
||||
return true;
|
||||
}
|
||||
|
||||
for (const auto & child : ptr->children)
|
||||
if (cannotBeMoved(child.get()))
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
void determineArrayJoinedNames(ASTSelectQuery & select)
|
||||
{
|
||||
/// much simplified code from ExpressionAnalyzer::getArrayJoinedColumns()
|
||||
if (!select.array_join_expression_list)
|
||||
return;
|
||||
|
||||
for (const auto & ast : select.array_join_expression_list->children)
|
||||
array_joined_names.emplace(ast->getAliasOrColumnName());
|
||||
}
|
||||
|
||||
string_set_t primary_key_columns{};
|
||||
string_set_t table_columns{};
|
||||
Logger * log;
|
||||
std::unordered_map<std::string, std::size_t> column_sizes{};
|
||||
std::size_t total_column_size{};
|
||||
NameSet array_joined_names;
|
||||
};
|
||||
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user