This commit is contained in:
Vsevolod Orlov 2015-06-15 17:00:05 +03:00
commit e34a1fcbca
202 changed files with 6228 additions and 1720 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -51,6 +51,7 @@
M(DelayedInserts) \
M(RejectedInserts) \
M(DelayedInsertsMilliseconds) \
M(SynchronousMergeOnInsert) \
\
M(ZooKeeperInit) \
M(ZooKeeperTransactions) \

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -35,7 +35,7 @@ public:
{
}
String getName() const override { return "AddingDefaultBlockInputStream"; }
String getName() const override { return "AddingDefault"; }
String getID() const override
{

View File

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

View File

@ -26,7 +26,7 @@ public:
{
}
String getName() const override { return "AggregatingSortedBlockInputStream"; }
String getName() const override { return "AggregatingSorted"; }
String getID() const override
{

View File

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

View File

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

View File

@ -23,7 +23,7 @@ public:
~CollapsingFinalBlockInputStream();
String getName() const override { return "CollapsingFinalBlockInputStream"; }
String getName() const override { return "CollapsingFinal"; }
String getID() const override
{

View File

@ -30,7 +30,7 @@ public:
{
}
String getName() const override { return "CollapsingSortedBlockInputStream"; }
String getName() const override { return "CollapsingSorted"; }
String getID() const override
{

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -51,7 +51,6 @@ public:
/** Для вывода дерева преобразований потока данных (плана выполнения запроса).
*/
virtual String getName() const = 0;
virtual String getShortName() const; /// То же самое, но без BlockInputStream на конце.
/** Уникальный идентификатор части конвейера выполнения запроса.
* Источники с одинаковым идентификатором считаются идентичными

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -61,7 +61,7 @@ public:
}
String getName() const override { return "RemoteBlockInputStream"; }
String getName() const override { return "Remote"; }
String getID() const override

View File

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

View File

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

View File

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

View File

@ -36,7 +36,7 @@ public:
children = inputs;
}
String getName() const override { return "UnionBlockInputStream"; }
String getName() const override { return "Union"; }
String getID() const override
{

View File

@ -13,7 +13,7 @@ using Poco::SharedPtr;
/** Тип - состояние агрегатной функции.
* Параметры типа - это агрегатная функция, типы её аргументов и её параметры (для параметрических агрегатных функций).
*/
class DataTypeAggregateFunction : public IDataType
class DataTypeAggregateFunction final : public IDataType
{
private:
AggregateFunctionPtr function;

View File

@ -9,7 +9,7 @@ namespace DB
using Poco::SharedPtr;
class DataTypeArray : public IDataType
class DataTypeArray final : public IDataType
{
private:
/// Тип элементов массивов.

View File

@ -10,7 +10,7 @@
namespace DB
{
class DataTypeDate : public IDataTypeNumberFixed<UInt16, ColumnUInt16>
class DataTypeDate final : public IDataTypeNumberFixed<UInt16, ColumnUInt16>
{
public:
DataTypeDate() {}

View File

@ -10,7 +10,7 @@
namespace DB
{
class DataTypeDateTime : public IDataTypeNumberFixed<UInt32, ColumnUInt32>
class DataTypeDateTime final : public IDataTypeNumberFixed<UInt32, ColumnUInt32>
{
public:
DataTypeDateTime() {}

View File

@ -9,7 +9,7 @@ namespace DB
/**
* Лямбда-выражение.
*/
class DataTypeExpression : public IDataTypeDummy
class DataTypeExpression final : public IDataTypeDummy
{
private:
DataTypes argument_types;

View File

@ -13,7 +13,7 @@ namespace DB
using Poco::SharedPtr;
class DataTypeFixedString : public IDataType
class DataTypeFixedString final : public IDataType
{
private:
size_t n;

View File

@ -9,7 +9,7 @@ namespace DB
using Poco::SharedPtr;
class DataTypeNested : public IDataType
class DataTypeNested final : public IDataType
{
private:
/// Имена и типы вложенных массивов.

View File

@ -9,7 +9,7 @@ namespace DB
/** Тип данных, соответствующий множеству значений в секции IN.
* Используется только как промежуточный вариант при вычислении выражений.
*/
class DataTypeSet : public IDataTypeDummy
class DataTypeSet final : public IDataTypeDummy
{
public:
std::string getName() const { return "Set"; }

View File

@ -13,7 +13,7 @@ namespace DB
using Poco::SharedPtr;
class DataTypeString : public IDataType
class DataTypeString final : public IDataType
{
public:
using FieldType = String;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -90,7 +90,7 @@ public:
}
}
String getName() const override { return "MySQLBlockInputStream"; }
String getName() const override { return "MySQL"; }
String getID() const override
{

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1,10 +1,9 @@
#pragma once
#include <Poco/Net/DNS.h>
#include <Yandex/Revision.h>
#include <math.h>
#include <mutex>
#include <DB/Core/Defines.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
@ -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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -130,6 +130,15 @@ private:
void ignoreWithTotals();
/** Если в запросе SELECT есть секция SETTINGS, то применить настройки из неё.
* Затем достать настройки из context и поместить их в settings.
*
* Секция SETTINGS - настройки для конкретного запроса.
* Обычно настройки могут быть переданы другими способами, не внутри запроса.
* Но использование такой секции оправдано, если нужно задать настройки для одного подзапроса.
*/
void initSettings();
ASTPtr query_ptr;
ASTSelectQuery & query;
Context context;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -9,11 +9,31 @@
namespace DB
{
/** SELECT запрос
*/
class ASTSelectQuery : public ASTQueryWithOutput
{
public:
ASTSelectQuery() = default;
ASTSelectQuery(const StringRange range_);
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "SelectQuery"; };
/// Проверить наличие функции arrayJoin. (Не большого ARRAY JOIN.)
static bool hasArrayJoin(const ASTPtr & ast);
/// Содержит ли запрос астериск?
bool hasAsterisk() const;
/// Переименовать столбцы запроса в такие же имена, как в исходном запросе.
void renameColumns(const ASTSelectQuery & source);
/// Переписывает select_expression_list, чтобы вернуть только необходимые столбцы в правильном порядке.
void rewriteSelectExpressionList(const Names & column_names);
ASTPtr clone() const override;
public:
bool distinct = false;
ASTPtr select_expression_list;
@ -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;
}
};
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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