mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge
This commit is contained in:
commit
bd53190998
@ -8,8 +8,9 @@
|
|||||||
#include <string>
|
#include <string>
|
||||||
#include <Poco/File.h>
|
#include <Poco/File.h>
|
||||||
#include <DB/Common/escapeForFileName.h>
|
#include <DB/Common/escapeForFileName.h>
|
||||||
#include <jsonxx.h>
|
|
||||||
#include <fstream>
|
#include <boost/property_tree/ptree.hpp>
|
||||||
|
#include <boost/property_tree/json_parser.hpp>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -20,13 +21,13 @@ class FileChecker
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
FileChecker(const std::string &file_info_path_, Storage & storage_) :
|
FileChecker(const std::string &file_info_path_, Storage & storage_) :
|
||||||
files_info_path(file_info_path_), storage(storage_), log(&Logger::get("FileChecker"))
|
files_info_path(file_info_path_), files_info(), storage(storage_), log(&Logger::get("FileChecker"))
|
||||||
{
|
{
|
||||||
Poco::Path path(files_info_path);
|
Poco::Path path(files_info_path);
|
||||||
tmp_files_info_path = path.parent().toString() + "tmp_" + path.getFileName();
|
tmp_files_info_path = path.parent().toString() + "tmp_" + path.getFileName();
|
||||||
|
|
||||||
std::ifstream istr(files_info_path);
|
if (Poco::File(files_info_path).exists())
|
||||||
files_info.parse(istr);
|
boost::property_tree::read_json(files_info_path, files_info);
|
||||||
}
|
}
|
||||||
|
|
||||||
void setPath(const std::string & file_info_path_)
|
void setPath(const std::string & file_info_path_)
|
||||||
@ -53,47 +54,46 @@ public:
|
|||||||
bool check() const
|
bool check() const
|
||||||
{
|
{
|
||||||
bool correct = true;
|
bool correct = true;
|
||||||
for (auto & node : files_info.kv_map())
|
if (!files_info.empty())
|
||||||
{
|
for (auto & node : files_info.get_child("yandex"))
|
||||||
std::string filename = unescapeForFileName(node.first);
|
|
||||||
size_t expected_size = std::stoull(node.second->get<jsonxx::Object>().get<std::string>("size"));
|
|
||||||
|
|
||||||
Poco::File file(Poco::Path(files_info_path).parent().toString() + "/" + filename);
|
|
||||||
if (!file.exists())
|
|
||||||
{
|
{
|
||||||
LOG_ERROR(log, "File " << file.path() << " doesn't exists");
|
std::string filename = unescapeForFileName(node.first);
|
||||||
correct = false;
|
size_t expected_size = std::stoull(node.second.get<std::string>("size"));
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
size_t real_size = file.getSize();
|
Poco::File file(Poco::Path(files_info_path).parent().toString() + "/" + filename);
|
||||||
if (real_size != expected_size)
|
if (!file.exists())
|
||||||
{
|
{
|
||||||
LOG_ERROR(log, "Size of " << file.path() << " is wrong. Size is " << real_size << " but should be " << expected_size);
|
LOG_ERROR(log, "File " << file.path() << " doesn't exists");
|
||||||
correct = false;
|
correct = false;
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t real_size = file.getSize();
|
||||||
|
if (real_size != expected_size)
|
||||||
|
{
|
||||||
|
LOG_ERROR(log, "Size of " << file.path() << " is wrong. Size is " << real_size << " but should be " << expected_size);
|
||||||
|
correct = false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
return correct;
|
return correct;
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
void updateTree(const Poco::File & file)
|
void updateTree(const Poco::File & file)
|
||||||
{
|
{
|
||||||
files_info.import(escapeForFileName(Poco::Path(file.path()).getFileName()),
|
files_info.put(std::string("yandex.") + escapeForFileName(Poco::Path(file.path()).getFileName()) + ".size", std::to_string(file.getSize()));
|
||||||
jsonxx::Object("size", std::to_string(file.getSize())));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void saveTree()
|
void saveTree()
|
||||||
{
|
{
|
||||||
std::ofstream file(tmp_files_info_path, std::ofstream::trunc);
|
boost::property_tree::write_json(tmp_files_info_path, files_info, std::locale());
|
||||||
file << files_info.write(jsonxx::JSON);
|
|
||||||
file.close();
|
|
||||||
|
|
||||||
std::string old_file_name = files_info_path + ".old";
|
Poco::File current_file(files_info_path);
|
||||||
Poco::File new_file(files_info_path);
|
|
||||||
if (new_file.exists())
|
if (current_file.exists())
|
||||||
{
|
{
|
||||||
new_file.renameTo(old_file_name);
|
std::string old_file_name = files_info_path + ".old";
|
||||||
|
current_file.renameTo(old_file_name);
|
||||||
Poco::File(tmp_files_info_path).renameTo(files_info_path);
|
Poco::File(tmp_files_info_path).renameTo(files_info_path);
|
||||||
Poco::File(old_file_name).remove();
|
Poco::File(old_file_name).remove();
|
||||||
}
|
}
|
||||||
@ -104,7 +104,8 @@ private:
|
|||||||
std::string files_info_path;
|
std::string files_info_path;
|
||||||
std::string tmp_files_info_path;
|
std::string tmp_files_info_path;
|
||||||
|
|
||||||
jsonxx::Object files_info;
|
using PropertyTree = boost::property_tree::ptree;
|
||||||
|
PropertyTree files_info;
|
||||||
|
|
||||||
Storage & storage;
|
Storage & storage;
|
||||||
Logger * log;
|
Logger * log;
|
||||||
|
@ -21,7 +21,7 @@
|
|||||||
* - ищем её в хэш-таблице, если нашли - достаём смещение из хэш-таблицы и сравниваем строку побайтово;
|
* - ищем её в хэш-таблице, если нашли - достаём смещение из хэш-таблицы и сравниваем строку побайтово;
|
||||||
* - если сравнить не получилось - проверяем следующую ячейку хэш-таблицы из цепочки разрешения коллизий;
|
* - если сравнить не получилось - проверяем следующую ячейку хэш-таблицы из цепочки разрешения коллизий;
|
||||||
* - если не нашли, пропускаем в haystack почти размер needle байт;
|
* - если не нашли, пропускаем в haystack почти размер needle байт;
|
||||||
*
|
*
|
||||||
* Используется невыровненный доступ к памяти.
|
* Используется невыровненный доступ к памяти.
|
||||||
*/
|
*/
|
||||||
class Volnitsky
|
class Volnitsky
|
||||||
@ -35,7 +35,7 @@ private:
|
|||||||
const char * needle_end;
|
const char * needle_end;
|
||||||
size_t step; /// Насколько двигаемся, если n-грамма из haystack не нашлась в хэш-таблице.
|
size_t step; /// Насколько двигаемся, если n-грамма из haystack не нашлась в хэш-таблице.
|
||||||
|
|
||||||
static const size_t hash_size = 64 * 1024; /// Обычно помещается в L1-кэш, хотя занимает его целиком.
|
static const size_t hash_size = 64 * 1024; /// Помещается в L2-кэш.
|
||||||
offset_t hash[hash_size]; /// Хэш-таблица.
|
offset_t hash[hash_size]; /// Хэш-таблица.
|
||||||
|
|
||||||
bool fallback; /// Нужно ли использовать fallback алгоритм.
|
bool fallback; /// Нужно ли использовать fallback алгоритм.
|
||||||
@ -57,7 +57,7 @@ public:
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
fallback = false;
|
fallback = false;
|
||||||
|
|
||||||
memset(hash, 0, hash_size * sizeof(hash[0]));
|
memset(hash, 0, hash_size * sizeof(hash[0]));
|
||||||
|
|
||||||
for (int i = needle_size - sizeof(ngram_t); i >= 0; --i)
|
for (int i = needle_size - sizeof(ngram_t); i >= 0; --i)
|
||||||
|
@ -63,7 +63,6 @@ public:
|
|||||||
BlockInputStreams & getChildren() { return children; }
|
BlockInputStreams & getChildren() { return children; }
|
||||||
|
|
||||||
void dumpTree(std::ostream & ostr, size_t indent = 0, size_t multiplier = 1);
|
void dumpTree(std::ostream & ostr, size_t indent = 0, size_t multiplier = 1);
|
||||||
void dumpTreeWithProfile(std::ostream & ostr, size_t indent = 0);
|
|
||||||
|
|
||||||
/// Получить листовые источники (не считая этот).
|
/// Получить листовые источники (не считая этот).
|
||||||
BlockInputStreams getLeaves();
|
BlockInputStreams getLeaves();
|
||||||
|
@ -21,8 +21,7 @@ namespace DB
|
|||||||
struct BlockStreamProfileInfo
|
struct BlockStreamProfileInfo
|
||||||
{
|
{
|
||||||
bool started = false;
|
bool started = false;
|
||||||
Stopwatch work_stopwatch; /// Время вычислений (выполнения функции read())
|
Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; /// Время с учётом ожидания
|
||||||
Stopwatch total_stopwatch; /// Время с учётом ожидания
|
|
||||||
|
|
||||||
String stream_name; /// Короткое имя потока, для которого собирается информация
|
String stream_name; /// Короткое имя потока, для которого собирается информация
|
||||||
|
|
||||||
@ -48,7 +47,6 @@ struct BlockStreamProfileInfo
|
|||||||
bool hasAppliedLimit() const;
|
bool hasAppliedLimit() const;
|
||||||
|
|
||||||
void update(Block & block);
|
void update(Block & block);
|
||||||
void print(std::ostream & ostr) const;
|
|
||||||
|
|
||||||
/// Методы для бинарной [де]сериализации
|
/// Методы для бинарной [де]сериализации
|
||||||
void read(ReadBuffer & in);
|
void read(ReadBuffer & in);
|
||||||
|
@ -1,7 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Poco/SharedPtr.h>
|
#include <Poco/SharedPtr.h>
|
||||||
|
|
||||||
#include <DB/Functions/IFunction.h>
|
#include <DB/Functions/IFunction.h>
|
||||||
|
|
||||||
|
|
||||||
@ -17,10 +16,19 @@ class Context;
|
|||||||
*/
|
*/
|
||||||
class FunctionFactory
|
class FunctionFactory
|
||||||
{
|
{
|
||||||
|
private:
|
||||||
|
typedef IFunction* (*Creator)(const Context & context); /// Не std::function, так как меньше indirection и размер объекта.
|
||||||
|
std::unordered_map<String, Creator> functions;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
FunctionPtr get(
|
FunctionFactory();
|
||||||
const String & name,
|
|
||||||
const Context & context) const;
|
FunctionPtr get(const String & name, const Context & context) const;
|
||||||
|
|
||||||
|
void registerFunction(const String & name, Creator creator)
|
||||||
|
{
|
||||||
|
functions[name] = creator;
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -4,10 +4,12 @@
|
|||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||||
|
#include <DB/DataTypes/DataTypeArray.h>
|
||||||
#include <DB/DataTypes/DataTypeDate.h>
|
#include <DB/DataTypes/DataTypeDate.h>
|
||||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
#include <DB/Columns/ColumnFixedString.h>
|
#include <DB/Columns/ColumnFixedString.h>
|
||||||
|
#include <DB/Columns/ColumnArray.h>
|
||||||
#include <DB/Columns/ColumnConst.h>
|
#include <DB/Columns/ColumnConst.h>
|
||||||
#include <DB/Functions/IFunction.h>
|
#include <DB/Functions/IFunction.h>
|
||||||
|
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#include <DB/Functions/IFunction.h>
|
#include <DB/Functions/IFunction.h>
|
||||||
#include "NumberTraits.h"
|
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/IO/WriteBufferFromVector.h>
|
#include <DB/IO/WriteBufferFromVector.h>
|
||||||
#include <DB/IO/WriteBufferFromString.h>
|
#include <DB/IO/WriteBufferFromString.h>
|
||||||
|
@ -1,8 +1,8 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/DataTypes/DataTypeString.h>
|
|
||||||
#include <DB/DataTypes/DataTypeArray.h>
|
#include <DB/DataTypes/DataTypeArray.h>
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
|
#include <DB/Columns/ColumnFixedString.h>
|
||||||
#include <DB/Columns/ColumnConst.h>
|
#include <DB/Columns/ColumnConst.h>
|
||||||
#include <DB/Columns/ColumnArray.h>
|
#include <DB/Columns/ColumnArray.h>
|
||||||
#include <DB/Functions/IFunction.h>
|
#include <DB/Functions/IFunction.h>
|
||||||
|
@ -7,7 +7,9 @@
|
|||||||
|
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
|
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
|
#include <DB/Columns/ColumnFixedString.h>
|
||||||
#include <DB/Columns/ColumnConst.h>
|
#include <DB/Columns/ColumnConst.h>
|
||||||
#include <DB/Common/Volnitsky.h>
|
#include <DB/Common/Volnitsky.h>
|
||||||
#include <DB/Functions/IFunction.h>
|
#include <DB/Functions/IFunction.h>
|
||||||
|
@ -91,7 +91,7 @@ struct AggregationMethodKey64
|
|||||||
|
|
||||||
/** Разместить дополнительные данные, если это необходимо, в случае, когда в хэш-таблицу был вставлен новый ключ.
|
/** Разместить дополнительные данные, если это необходимо, в случае, когда в хэш-таблицу был вставлен новый ключ.
|
||||||
*/
|
*/
|
||||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -139,7 +139,7 @@ struct AggregationMethodString
|
|||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||||
|
|
||||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||||
{
|
{
|
||||||
it->first.data = pool.insert(it->first.data, it->first.size);
|
it->first.data = pool.insert(it->first.data, it->first.size);
|
||||||
}
|
}
|
||||||
@ -186,7 +186,7 @@ struct AggregationMethodFixedString
|
|||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||||
|
|
||||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||||
{
|
{
|
||||||
it->first.data = pool.insert(it->first.data, it->first.size);
|
it->first.data = pool.insert(it->first.data, it->first.size);
|
||||||
}
|
}
|
||||||
@ -226,7 +226,7 @@ struct AggregationMethodKeys128
|
|||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||||
|
|
||||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -271,7 +271,7 @@ struct AggregationMethodHashed
|
|||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value.second; }
|
static AggregateDataPtr & getAggregateData(Mapped & value) { return value.second; }
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value.second; }
|
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value.second; }
|
||||||
|
|
||||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||||
{
|
{
|
||||||
it->second.first = placeKeysInPool(i, keys_size, keys, pool);
|
it->second.first = placeKeysInPool(i, keys_size, keys, pool);
|
||||||
}
|
}
|
||||||
@ -308,7 +308,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
|
|
||||||
size_t keys_size; /// Количество ключей NOTE нужно ли это поле?
|
size_t keys_size; /// Количество ключей NOTE нужно ли это поле?
|
||||||
Sizes key_sizes; /// Размеры ключей, если ключи фиксированной длины
|
Sizes key_sizes; /// Размеры ключей, если ключи фиксированной длины
|
||||||
|
|
||||||
/// Пулы для состояний агрегатных функций. Владение потом будет передано в ColumnAggregateFunction.
|
/// Пулы для состояний агрегатных функций. Владение потом будет передано в ColumnAggregateFunction.
|
||||||
Arenas aggregates_pools;
|
Arenas aggregates_pools;
|
||||||
Arena * aggregates_pool; /// Пул, который сейчас используется для аллокации.
|
Arena * aggregates_pool; /// Пул, который сейчас используется для аллокации.
|
||||||
@ -322,7 +322,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
std::unique_ptr<AggregationMethodFixedString> key_fixed_string;
|
std::unique_ptr<AggregationMethodFixedString> key_fixed_string;
|
||||||
std::unique_ptr<AggregationMethodKeys128> keys128;
|
std::unique_ptr<AggregationMethodKeys128> keys128;
|
||||||
std::unique_ptr<AggregationMethodHashed> hashed;
|
std::unique_ptr<AggregationMethodHashed> hashed;
|
||||||
|
|
||||||
enum Type
|
enum Type
|
||||||
{
|
{
|
||||||
EMPTY = 0,
|
EMPTY = 0,
|
||||||
@ -465,7 +465,7 @@ public:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
friend struct AggregatedDataVariants;
|
friend struct AggregatedDataVariants;
|
||||||
|
|
||||||
ColumnNumbers keys;
|
ColumnNumbers keys;
|
||||||
Names key_names;
|
Names key_names;
|
||||||
AggregateDescriptions aggregates;
|
AggregateDescriptions aggregates;
|
||||||
|
@ -34,7 +34,7 @@ public:
|
|||||||
/// contains names of directories for asynchronous write to StorageDistributed
|
/// contains names of directories for asynchronous write to StorageDistributed
|
||||||
std::vector<std::string> dir_names;
|
std::vector<std::string> dir_names;
|
||||||
int weight;
|
int weight;
|
||||||
bool has_local_node;
|
size_t num_local_nodes;
|
||||||
};
|
};
|
||||||
std::vector<ShardInfo> shard_info_vec;
|
std::vector<ShardInfo> shard_info_vec;
|
||||||
std::vector<size_t> slot_to_shard;
|
std::vector<size_t> slot_to_shard;
|
||||||
|
@ -57,8 +57,8 @@ typedef std::vector<DatabaseAndTableName> Dependencies;
|
|||||||
*/
|
*/
|
||||||
struct ContextShared
|
struct ContextShared
|
||||||
{
|
{
|
||||||
Logger * log; /// Логгер.
|
Logger * log = &Logger::get("Context"); /// Логгер.
|
||||||
|
|
||||||
struct AfterDestroy
|
struct AfterDestroy
|
||||||
{
|
{
|
||||||
Logger * log;
|
Logger * log;
|
||||||
@ -70,7 +70,7 @@ struct ContextShared
|
|||||||
LOG_INFO(log, "Uninitialized shared context.");
|
LOG_INFO(log, "Uninitialized shared context.");
|
||||||
#endif
|
#endif
|
||||||
}
|
}
|
||||||
} after_destroy;
|
} after_destroy {log};
|
||||||
|
|
||||||
mutable Poco::Mutex mutex; /// Для доступа и модификации разделяемых объектов.
|
mutable Poco::Mutex mutex; /// Для доступа и модификации разделяемых объектов.
|
||||||
|
|
||||||
@ -96,7 +96,6 @@ struct ContextShared
|
|||||||
ViewDependencies view_dependencies; /// Текущие зависимости
|
ViewDependencies view_dependencies; /// Текущие зависимости
|
||||||
ConfigurationPtr users_config; /// Конфиг с секциями users, profiles и quotas.
|
ConfigurationPtr users_config; /// Конфиг с секциями users, profiles и quotas.
|
||||||
InterserverIOHandler interserver_io_handler; /// Обработчик для межсерверной передачи данных.
|
InterserverIOHandler interserver_io_handler; /// Обработчик для межсерверной передачи данных.
|
||||||
String default_replica_name; /// Имя реплики из конфига. DEPRECATED
|
|
||||||
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
|
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
|
||||||
Macros macros; /// Подстановки из конфига.
|
Macros macros; /// Подстановки из конфига.
|
||||||
|
|
||||||
@ -107,8 +106,6 @@ struct ContextShared
|
|||||||
bool shutdown_called = false;
|
bool shutdown_called = false;
|
||||||
|
|
||||||
|
|
||||||
ContextShared() : log(&Logger::get("Context")), after_destroy(log) {};
|
|
||||||
|
|
||||||
~ContextShared()
|
~ContextShared()
|
||||||
{
|
{
|
||||||
#ifndef DBMS_CLIENT
|
#ifndef DBMS_CLIENT
|
||||||
@ -229,7 +226,7 @@ public:
|
|||||||
|
|
||||||
/// Возвращает отцепленную таблицу.
|
/// Возвращает отцепленную таблицу.
|
||||||
StoragePtr detachTable(const String & database_name, const String & table_name);
|
StoragePtr detachTable(const String & database_name, const String & table_name);
|
||||||
|
|
||||||
void detachDatabase(const String & database_name);
|
void detachDatabase(const String & database_name);
|
||||||
|
|
||||||
String getCurrentDatabase() const;
|
String getCurrentDatabase() const;
|
||||||
@ -240,10 +237,6 @@ public:
|
|||||||
String getDefaultFormat() const; /// Если default_format не задан - возвращается некоторый глобальный формат по-умолчанию.
|
String getDefaultFormat() const; /// Если default_format не задан - возвращается некоторый глобальный формат по-умолчанию.
|
||||||
void setDefaultFormat(const String & name);
|
void setDefaultFormat(const String & name);
|
||||||
|
|
||||||
/// Имя этой реплики из конфига.
|
|
||||||
String getDefaultReplicaName() const;
|
|
||||||
void setDefaultReplicaName(const String & name);
|
|
||||||
|
|
||||||
const Macros & getMacros() const;
|
const Macros & getMacros() const;
|
||||||
void setMacros(Macros && macros);
|
void setMacros(Macros && macros);
|
||||||
|
|
||||||
|
@ -1,7 +1,5 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <statdaemons/Stopwatch.h>
|
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <Yandex/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Parsers/ASTJoin.h>
|
#include <DB/Parsers/ASTJoin.h>
|
||||||
|
@ -2,8 +2,6 @@
|
|||||||
|
|
||||||
#include <set>
|
#include <set>
|
||||||
|
|
||||||
#include <statdaemons/Stopwatch.h>
|
|
||||||
|
|
||||||
#include <Yandex/logger_useful.h>
|
#include <Yandex/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Core/ColumnNumbers.h>
|
#include <DB/Core/ColumnNumbers.h>
|
||||||
|
@ -20,7 +20,7 @@ namespace
|
|||||||
|
|
||||||
for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it)
|
for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it)
|
||||||
{
|
{
|
||||||
const auto & address = boost::copy_range<std::string>(*it);
|
const auto address = boost::copy_range<std::string>(*it);
|
||||||
|
|
||||||
const auto user_pw_end = strchr(address.data(), '@');
|
const auto user_pw_end = strchr(address.data(), '@');
|
||||||
const auto colon = strchr(address.data(), ':');
|
const auto colon = strchr(address.data(), ':');
|
||||||
@ -65,8 +65,8 @@ public:
|
|||||||
~DirectoryMonitor()
|
~DirectoryMonitor()
|
||||||
{
|
{
|
||||||
{
|
{
|
||||||
std::lock_guard<std::mutex> lock{mutex};
|
|
||||||
quit = true;
|
quit = true;
|
||||||
|
std::lock_guard<std::mutex> lock{mutex};
|
||||||
}
|
}
|
||||||
cond.notify_one();
|
cond.notify_one();
|
||||||
thread.join();
|
thread.join();
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
#include <statdaemons/stdext.h>
|
#include <statdaemons/stdext.h>
|
||||||
|
|
||||||
#include <iostream>
|
#include <iostream>
|
||||||
|
#include <type_traits>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -41,56 +42,117 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
template <typename T>
|
||||||
|
static std::vector<IColumn::Filter> createFiltersImpl(const size_t num_rows, const IColumn * column, const Cluster & cluster)
|
||||||
|
{
|
||||||
|
const auto total_weight = cluster.slot_to_shard.size();
|
||||||
|
const auto num_shards = cluster.shard_info_vec.size();
|
||||||
|
std::vector<IColumn::Filter> filters(num_shards);
|
||||||
|
|
||||||
|
/** Деление отрицательного числа с остатком на положительное, в C++ даёт отрицательный остаток.
|
||||||
|
* Для данной задачи это не подходит. Поэтому, будем обрабатывать знаковые типы как беззнаковые.
|
||||||
|
* Это даёт уже что-то совсем не похожее на деление с остатком, но подходящее для данной задачи.
|
||||||
|
*/
|
||||||
|
using UnsignedT = typename std::make_unsigned<T>::type;
|
||||||
|
|
||||||
|
/// const columns contain only one value, therefore we do not need to read it at every iteration
|
||||||
|
if (column->isConst())
|
||||||
|
{
|
||||||
|
const auto data = typeid_cast<const ColumnConst<T> *>(column)->getData();
|
||||||
|
const auto shard_num = cluster.slot_to_shard[static_cast<UnsignedT>(data) % total_weight];
|
||||||
|
|
||||||
|
for (size_t i = 0; i < num_shards; ++i)
|
||||||
|
filters[i].assign(num_rows, static_cast<UInt8>(shard_num == i));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
const auto & data = typeid_cast<const ColumnVector<T> *>(column)->getData();
|
||||||
|
|
||||||
|
for (size_t i = 0; i < num_shards; ++i)
|
||||||
|
{
|
||||||
|
filters[i].resize(num_rows);
|
||||||
|
for (size_t j = 0; j < num_rows; ++j)
|
||||||
|
filters[i][j] = cluster.slot_to_shard[static_cast<UnsignedT>(data[j]) % total_weight] == i;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return filters;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<IColumn::Filter> createFilters(Block block)
|
||||||
|
{
|
||||||
|
using create_filters_sig = std::vector<IColumn::Filter>(size_t, const IColumn *, const Cluster &);
|
||||||
|
/// hashmap of pointers to functions corresponding to each integral type
|
||||||
|
static std::unordered_map<std::string, create_filters_sig *> creators{
|
||||||
|
{ TypeName<UInt8>::get(), &createFiltersImpl<UInt8> },
|
||||||
|
{ TypeName<UInt16>::get(), &createFiltersImpl<UInt16> },
|
||||||
|
{ TypeName<UInt32>::get(), &createFiltersImpl<UInt32> },
|
||||||
|
{ TypeName<UInt64>::get(), &createFiltersImpl<UInt64> },
|
||||||
|
{ TypeName<Int8>::get(), &createFiltersImpl<Int8> },
|
||||||
|
{ TypeName<Int16>::get(), &createFiltersImpl<Int16> },
|
||||||
|
{ TypeName<Int32>::get(), &createFiltersImpl<Int32> },
|
||||||
|
{ TypeName<Int64>::get(), &createFiltersImpl<Int64> },
|
||||||
|
};
|
||||||
|
|
||||||
|
storage.getShardingKeyExpr()->execute(block);
|
||||||
|
|
||||||
|
const auto & key_column = block.getByName(storage.getShardingKeyColumnName());
|
||||||
|
|
||||||
|
/// check that key column has valid type
|
||||||
|
const auto it = creators.find(key_column.type->getName());
|
||||||
|
|
||||||
|
return it != std::end(creators)
|
||||||
|
? (*it->second)(block.rowsInFirstColumn(), key_column.column.get(), storage.cluster)
|
||||||
|
: throw Exception{
|
||||||
|
"Sharding key expression does not evaluate to an integer type",
|
||||||
|
ErrorCodes::TYPE_MISMATCH
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
void writeSplit(const Block & block)
|
void writeSplit(const Block & block)
|
||||||
{
|
{
|
||||||
auto block_with_key = block;
|
|
||||||
storage.getShardingKeyExpr()->execute(block_with_key);
|
|
||||||
|
|
||||||
const auto & key_column = block_with_key.getByName(storage.getShardingKeyColumnName()).column;
|
|
||||||
const auto total_weight = storage.cluster.slot_to_shard.size();
|
|
||||||
|
|
||||||
/// shard => block mapping
|
|
||||||
std::vector<std::unique_ptr<Block>> target_blocks(storage.cluster.shard_info_vec.size());
|
|
||||||
|
|
||||||
const auto num_cols = block.columns();
|
const auto num_cols = block.columns();
|
||||||
|
/// cache column pointers for later reuse
|
||||||
std::vector<const IColumn*> columns(num_cols);
|
std::vector<const IColumn*> columns(num_cols);
|
||||||
for (size_t i = 0; i < columns.size(); ++i)
|
for (size_t i = 0; i < columns.size(); ++i)
|
||||||
columns[i] = block.getByPosition(i).column;
|
columns[i] = block.getByPosition(i).column;
|
||||||
|
|
||||||
for (size_t num_rows = block.rowsInFirstColumn(), row = 0; row < num_rows; ++row)
|
auto filters = createFilters(block);
|
||||||
|
|
||||||
|
const auto num_shards = storage.cluster.shard_info_vec.size();
|
||||||
|
for (size_t i = 0; i < num_shards; ++i)
|
||||||
{
|
{
|
||||||
const auto target_block_idx = storage.cluster.slot_to_shard[key_column->get64(row) % total_weight];
|
auto target_block = block.cloneEmpty();
|
||||||
auto & target_block = target_blocks[target_block_idx];
|
|
||||||
if (!target_block)
|
|
||||||
target_block = stdext::make_unique<Block>(block.cloneEmpty());
|
|
||||||
|
|
||||||
for (size_t col = 0; col < num_cols; ++col)
|
for (size_t col = 0; col < num_cols; ++col)
|
||||||
target_block->getByPosition(col).column->insertFrom(*columns[col], row);
|
target_block.getByPosition(col).column = columns[col]->filter(filters[i]);
|
||||||
}
|
|
||||||
|
|
||||||
for (size_t i = 0; i < target_blocks.size(); ++i)
|
if (target_block.rowsInFirstColumn())
|
||||||
if (const auto & target_block = target_blocks[i])
|
writeImpl(target_block, i);
|
||||||
writeImpl(*target_block, i);
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void writeImpl(const Block & block, const size_t shard_id = 0)
|
void writeImpl(const Block & block, const size_t shard_id = 0)
|
||||||
{
|
{
|
||||||
const auto & shard_info = storage.cluster.shard_info_vec[shard_id];
|
const auto & shard_info = storage.cluster.shard_info_vec[shard_id];
|
||||||
if (shard_info.has_local_node)
|
if (shard_info.num_local_nodes)
|
||||||
writeToLocal(block);
|
writeToLocal(block, shard_info.num_local_nodes);
|
||||||
|
|
||||||
/// dir_names is empty if shard has only local addresses
|
/// dir_names is empty if shard has only local addresses
|
||||||
if (!shard_info.dir_names.empty())
|
if (!shard_info.dir_names.empty())
|
||||||
writeToShard(block, shard_info.dir_names);
|
writeToShard(block, shard_info.dir_names);
|
||||||
}
|
}
|
||||||
|
|
||||||
void writeToLocal(const Block & block)
|
void writeToLocal(const Block & block, const size_t repeats)
|
||||||
{
|
{
|
||||||
InterpreterInsertQuery interp{query_ast, storage.context};
|
InterpreterInsertQuery interp{query_ast, storage.context};
|
||||||
|
|
||||||
auto block_io = interp.execute();
|
auto block_io = interp.execute();
|
||||||
block_io.out->writePrefix();
|
block_io.out->writePrefix();
|
||||||
block_io.out->write(block);
|
|
||||||
|
for (size_t i = 0; i < repeats; ++i)
|
||||||
|
block_io.out->write(block);
|
||||||
|
|
||||||
block_io.out->writeSuffix();
|
block_io.out->writeSuffix();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -102,7 +164,7 @@ private:
|
|||||||
std::string first_file_tmp_path{};
|
std::string first_file_tmp_path{};
|
||||||
|
|
||||||
auto first = true;
|
auto first = true;
|
||||||
const auto & query_string = queryToString<ASTInsertQuery>(query_ast);
|
const auto & query_string = queryToString(query_ast);
|
||||||
|
|
||||||
/// write first file, hardlink the others
|
/// write first file, hardlink the others
|
||||||
for (const auto & dir_name : dir_names)
|
for (const auto & dir_name : dir_names)
|
||||||
|
@ -4,13 +4,10 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
template <typename ASTType>
|
|
||||||
inline std::string queryToString(const ASTPtr & query)
|
inline std::string queryToString(const ASTPtr & query)
|
||||||
{
|
{
|
||||||
const auto & query_ast = typeid_cast<const ASTType &>(*query);
|
|
||||||
|
|
||||||
std::ostringstream s;
|
std::ostringstream s;
|
||||||
formatAST(query_ast, s, 0, false, true);
|
formatAST(*query, s, 0, false, true);
|
||||||
|
|
||||||
return s.str();
|
return s.str();
|
||||||
}
|
}
|
||||||
|
@ -3,7 +3,6 @@
|
|||||||
#include <Poco/SharedPtr.h>
|
#include <Poco/SharedPtr.h>
|
||||||
|
|
||||||
#include <DB/Storages/IStorage.h>
|
#include <DB/Storages/IStorage.h>
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -12,20 +11,6 @@ namespace DB
|
|||||||
using Poco::SharedPtr;
|
using Poco::SharedPtr;
|
||||||
|
|
||||||
|
|
||||||
class NumbersBlockInputStream : public IProfilingBlockInputStream
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
NumbersBlockInputStream(size_t block_size_);
|
|
||||||
String getName() const { return "NumbersBlockInputStream"; }
|
|
||||||
String getID() const { return "Numbers"; }
|
|
||||||
protected:
|
|
||||||
Block readImpl();
|
|
||||||
private:
|
|
||||||
size_t block_size;
|
|
||||||
UInt64 next;
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
/** Реализует хранилище для системной таблицы Numbers.
|
/** Реализует хранилище для системной таблицы Numbers.
|
||||||
* Таблица содержит единственный столбец number UInt64.
|
* Таблица содержит единственный столбец number UInt64.
|
||||||
* Из этой таблицы можно прочитать все натуральные числа, начиная с 0 (до 2^64 - 1, а потом заново).
|
* Из этой таблицы можно прочитать все натуральные числа, начиная с 0 (до 2^64 - 1, а потом заново).
|
||||||
@ -33,7 +18,7 @@ private:
|
|||||||
class StorageSystemNumbers : public IStorage
|
class StorageSystemNumbers : public IStorage
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
static StoragePtr create(const std::string & name_);
|
static StoragePtr create(const std::string & name_, bool multithreaded_ = false);
|
||||||
|
|
||||||
std::string getName() const { return "SystemNumbers"; }
|
std::string getName() const { return "SystemNumbers"; }
|
||||||
std::string getTableName() const { return name; }
|
std::string getTableName() const { return name; }
|
||||||
@ -51,8 +36,9 @@ public:
|
|||||||
private:
|
private:
|
||||||
const std::string name;
|
const std::string name;
|
||||||
NamesAndTypesList columns;
|
NamesAndTypesList columns;
|
||||||
|
bool multithreaded;
|
||||||
|
|
||||||
StorageSystemNumbers(const std::string & name_);
|
StorageSystemNumbers(const std::string & name_, bool multithreaded_);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -111,6 +111,8 @@ void CreatingSetsBlockInputStream::create(SubqueryForSet & subquery)
|
|||||||
size_t rows = 0;
|
size_t rows = 0;
|
||||||
size_t bytes = 0;
|
size_t bytes = 0;
|
||||||
|
|
||||||
|
watch.stop();
|
||||||
|
|
||||||
subquery.source->getLeafRowsBytes(rows, bytes);
|
subquery.source->getLeafRowsBytes(rows, bytes);
|
||||||
|
|
||||||
size_t head_rows = 0;
|
size_t head_rows = 0;
|
||||||
|
@ -91,31 +91,6 @@ void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t mult
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void IBlockInputStream::dumpTreeWithProfile(std::ostream & ostr, size_t indent)
|
|
||||||
{
|
|
||||||
ostr << indent + 1 << ". " << getShortName() << "." << std::endl;
|
|
||||||
|
|
||||||
/// Для красоты
|
|
||||||
size_t width = log10(indent + 1) + 4 + getShortName().size();
|
|
||||||
for (size_t i = 0; i < width; ++i)
|
|
||||||
ostr << "─";
|
|
||||||
ostr << std::endl;
|
|
||||||
|
|
||||||
/// Информация профайлинга, если есть
|
|
||||||
if (IProfilingBlockInputStream * profiling = dynamic_cast<IProfilingBlockInputStream *>(this))
|
|
||||||
{
|
|
||||||
if (profiling->getInfo().blocks != 0)
|
|
||||||
{
|
|
||||||
profiling->getInfo().print(ostr);
|
|
||||||
ostr << std::endl;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
|
|
||||||
(*it)->dumpTreeWithProfile(ostr, indent + 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
String IBlockInputStream::getShortName() const
|
String IBlockInputStream::getShortName() const
|
||||||
{
|
{
|
||||||
String res = getName();
|
String res = getName();
|
||||||
|
@ -18,7 +18,7 @@ void BlockStreamProfileInfo::read(ReadBuffer & in)
|
|||||||
readVarUInt(bytes, in);
|
readVarUInt(bytes, in);
|
||||||
readBinary(applied_limit, in);
|
readBinary(applied_limit, in);
|
||||||
readVarUInt(rows_before_limit, in);
|
readVarUInt(rows_before_limit, in);
|
||||||
readBinary(calculated_rows_before_limit, in);
|
readBinary(calculated_rows_before_limit, in);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -46,13 +46,13 @@ bool BlockStreamProfileInfo::hasAppliedLimit() const
|
|||||||
if (!calculated_rows_before_limit)
|
if (!calculated_rows_before_limit)
|
||||||
calculateRowsBeforeLimit();
|
calculateRowsBeforeLimit();
|
||||||
return applied_limit;
|
return applied_limit;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void BlockStreamProfileInfo::update(Block & block)
|
void BlockStreamProfileInfo::update(Block & block)
|
||||||
{
|
{
|
||||||
++blocks;
|
++blocks;
|
||||||
rows += block.rows();
|
rows += block.rowsInFirstColumn();
|
||||||
bytes += block.bytes();
|
bytes += block.bytes();
|
||||||
|
|
||||||
if (column_names.empty())
|
if (column_names.empty())
|
||||||
@ -76,7 +76,7 @@ void BlockStreamProfileInfo::collectInfosForStreamsWithName(const String & name,
|
|||||||
void BlockStreamProfileInfo::calculateRowsBeforeLimit() const
|
void BlockStreamProfileInfo::calculateRowsBeforeLimit() const
|
||||||
{
|
{
|
||||||
calculated_rows_before_limit = true;
|
calculated_rows_before_limit = true;
|
||||||
|
|
||||||
/// есть ли Limit?
|
/// есть ли Limit?
|
||||||
BlockStreamProfileInfos limits;
|
BlockStreamProfileInfos limits;
|
||||||
collectInfosForStreamsWithName("Limit", limits);
|
collectInfosForStreamsWithName("Limit", limits);
|
||||||
@ -99,63 +99,6 @@ void BlockStreamProfileInfo::calculateRowsBeforeLimit() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void BlockStreamProfileInfo::print(std::ostream & ostr) const
|
|
||||||
{
|
|
||||||
UInt64 elapsed = work_stopwatch.elapsed();
|
|
||||||
UInt64 nested_elapsed = 0;
|
|
||||||
double elapsed_seconds = work_stopwatch.elapsedSeconds();
|
|
||||||
double nested_elapsed_seconds = 0;
|
|
||||||
|
|
||||||
UInt64 nested_rows = 0;
|
|
||||||
UInt64 nested_blocks = 0;
|
|
||||||
UInt64 nested_bytes = 0;
|
|
||||||
|
|
||||||
if (!nested_infos.empty())
|
|
||||||
{
|
|
||||||
for (BlockStreamProfileInfos::const_iterator it = nested_infos.begin(); it != nested_infos.end(); ++it)
|
|
||||||
{
|
|
||||||
if ((*it)->work_stopwatch.elapsed() > nested_elapsed)
|
|
||||||
{
|
|
||||||
nested_elapsed = (*it)->work_stopwatch.elapsed();
|
|
||||||
nested_elapsed_seconds = (*it)->work_stopwatch.elapsedSeconds();
|
|
||||||
}
|
|
||||||
|
|
||||||
nested_rows += (*it)->rows;
|
|
||||||
nested_blocks += (*it)->blocks;
|
|
||||||
nested_bytes += (*it)->bytes;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
ostr << std::fixed << std::setprecision(2)
|
|
||||||
<< "Columns: " << column_names << std::endl
|
|
||||||
<< "Elapsed: " << elapsed_seconds << " sec. "
|
|
||||||
<< "(" << elapsed * 100.0 / total_stopwatch.elapsed() << "%), " << std::endl;
|
|
||||||
|
|
||||||
if (!nested_infos.empty())
|
|
||||||
{
|
|
||||||
double self_percents = (elapsed - nested_elapsed) * 100.0 / total_stopwatch.elapsed();
|
|
||||||
|
|
||||||
ostr<< "Elapsed (self): " << (elapsed_seconds - nested_elapsed_seconds) << " sec. "
|
|
||||||
<< "(" << (self_percents >= 50 ? "\033[1;31m" : (self_percents >= 10 ? "\033[1;33m" : "")) /// Раскраска больших значений
|
|
||||||
<< self_percents << "%"
|
|
||||||
<< (self_percents >= 10 ? "\033[0m" : "") << "), " << std::endl
|
|
||||||
<< "Rows (in): " << nested_rows << ", per second: " << nested_rows / elapsed_seconds << ", " << std::endl
|
|
||||||
<< "Blocks (in): " << nested_blocks << ", per second: " << nested_blocks / elapsed_seconds << ", " << std::endl
|
|
||||||
<< " " << nested_bytes / 1000000.0 << " MB (memory), "
|
|
||||||
<< nested_bytes * 1000 / elapsed << " MB/s (memory), " << std::endl;
|
|
||||||
|
|
||||||
if (self_percents > 0.1)
|
|
||||||
ostr << "Rows per second (in, self): " << (nested_rows / (elapsed_seconds - nested_elapsed_seconds))
|
|
||||||
<< ", " << (elapsed - nested_elapsed) / nested_rows << " ns/row, " << std::endl;
|
|
||||||
}
|
|
||||||
|
|
||||||
ostr << "Rows (out): " << rows << ", per second: " << rows / elapsed_seconds << ", " << std::endl
|
|
||||||
<< "Blocks (out): " << blocks << ", per second: " << blocks / elapsed_seconds << ", " << std::endl
|
|
||||||
<< " " << bytes / 1000000.0 << " MB (memory), " << bytes * 1000 / elapsed << " MB/s (memory), " << std::endl
|
|
||||||
<< "Average block size (out): " << rows / blocks << "." << std::endl;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
Block IProfilingBlockInputStream::read()
|
Block IProfilingBlockInputStream::read()
|
||||||
{
|
{
|
||||||
if (!info.started)
|
if (!info.started)
|
||||||
@ -166,7 +109,7 @@ Block IProfilingBlockInputStream::read()
|
|||||||
for (BlockInputStreams::const_iterator it = children.begin(); it != children.end(); ++it)
|
for (BlockInputStreams::const_iterator it = children.begin(); it != children.end(); ++it)
|
||||||
if (const IProfilingBlockInputStream * child = dynamic_cast<const IProfilingBlockInputStream *>(&**it))
|
if (const IProfilingBlockInputStream * child = dynamic_cast<const IProfilingBlockInputStream *>(&**it))
|
||||||
info.nested_infos.push_back(&child->info);
|
info.nested_infos.push_back(&child->info);
|
||||||
|
|
||||||
info.started = true;
|
info.started = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -175,9 +118,7 @@ Block IProfilingBlockInputStream::read()
|
|||||||
if (is_cancelled)
|
if (is_cancelled)
|
||||||
return res;
|
return res;
|
||||||
|
|
||||||
info.work_stopwatch.start();
|
|
||||||
res = readImpl();
|
res = readImpl();
|
||||||
info.work_stopwatch.stop();
|
|
||||||
|
|
||||||
/* if (res)
|
/* if (res)
|
||||||
{
|
{
|
||||||
@ -194,7 +135,7 @@ Block IProfilingBlockInputStream::read()
|
|||||||
std::cerr << ", ";
|
std::cerr << ", ";
|
||||||
std::cerr << res.getByPosition(i).name << " (" << res.getByPosition(i).column->size() << ")";
|
std::cerr << res.getByPosition(i).name << " (" << res.getByPosition(i).column->size() << ")";
|
||||||
}
|
}
|
||||||
|
|
||||||
std::cerr << std::endl;
|
std::cerr << std::endl;
|
||||||
}*/
|
}*/
|
||||||
|
|
||||||
@ -225,7 +166,7 @@ Block IProfilingBlockInputStream::read()
|
|||||||
cancel();
|
cancel();
|
||||||
}
|
}
|
||||||
|
|
||||||
progress(res.rows(), res.bytes());
|
progress(res.rowsInFirstColumn(), res.bytes());
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
@ -269,7 +210,7 @@ void IProfilingBlockInputStream::updateExtremes(Block & block)
|
|||||||
for (size_t i = 0; i < columns; ++i)
|
for (size_t i = 0; i < columns; ++i)
|
||||||
{
|
{
|
||||||
ColumnPtr & column = extremes.getByPosition(i).column;
|
ColumnPtr & column = extremes.getByPosition(i).column;
|
||||||
|
|
||||||
Field min_value = (*column)[0];
|
Field min_value = (*column)[0];
|
||||||
Field max_value = (*column)[1];
|
Field max_value = (*column)[1];
|
||||||
|
|
||||||
@ -330,9 +271,6 @@ bool IProfilingBlockInputStream::checkLimits()
|
|||||||
|
|
||||||
void IProfilingBlockInputStream::checkQuota(Block & block)
|
void IProfilingBlockInputStream::checkQuota(Block & block)
|
||||||
{
|
{
|
||||||
time_t current_time = time(0);
|
|
||||||
double total_elapsed = info.total_stopwatch.elapsedSeconds();
|
|
||||||
|
|
||||||
switch (limits.mode)
|
switch (limits.mode)
|
||||||
{
|
{
|
||||||
case LIMITS_TOTAL:
|
case LIMITS_TOTAL:
|
||||||
@ -340,15 +278,20 @@ void IProfilingBlockInputStream::checkQuota(Block & block)
|
|||||||
break;
|
break;
|
||||||
|
|
||||||
case LIMITS_CURRENT:
|
case LIMITS_CURRENT:
|
||||||
quota->checkAndAddResultRowsBytes(current_time, block.rows(), block.bytes());
|
{
|
||||||
|
time_t current_time = time(0);
|
||||||
|
double total_elapsed = info.total_stopwatch.elapsedSeconds();
|
||||||
|
|
||||||
|
quota->checkAndAddResultRowsBytes(current_time, block.rowsInFirstColumn(), block.bytes());
|
||||||
quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0));
|
quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0));
|
||||||
|
|
||||||
|
prev_elapsed = total_elapsed;
|
||||||
break;
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw Exception("Logical error: unknown limits mode.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: unknown limits mode.", ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
prev_elapsed = total_elapsed;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -366,10 +309,9 @@ void IProfilingBlockInputStream::progressImpl(size_t rows, size_t bytes)
|
|||||||
cancel();
|
cancel();
|
||||||
|
|
||||||
/// Общее количество данных, обработанных во всех листовых источниках, возможно, на удалённых серверах.
|
/// Общее количество данных, обработанных во всех листовых источниках, возможно, на удалённых серверах.
|
||||||
|
|
||||||
size_t total_rows = process_list_elem->rows_processed;
|
size_t total_rows = process_list_elem->rows_processed;
|
||||||
size_t total_bytes = process_list_elem->bytes_processed;
|
size_t total_bytes = process_list_elem->bytes_processed;
|
||||||
double total_elapsed = info.total_stopwatch.elapsedSeconds();
|
|
||||||
|
|
||||||
/** Проверяем ограничения на объём данных для чтения, скорость выполнения запроса, квоту на объём данных для чтения.
|
/** Проверяем ограничения на объём данных для чтения, скорость выполнения запроса, квоту на объём данных для чтения.
|
||||||
* NOTE: Может быть, имеет смысл сделать, чтобы они проверялись прямо в ProcessList?
|
* NOTE: Может быть, имеет смысл сделать, чтобы они проверялись прямо в ProcessList?
|
||||||
@ -389,13 +331,17 @@ void IProfilingBlockInputStream::progressImpl(size_t rows, size_t bytes)
|
|||||||
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (limits.min_execution_speed
|
if (limits.min_execution_speed)
|
||||||
&& total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0
|
|
||||||
&& total_rows / total_elapsed < limits.min_execution_speed)
|
|
||||||
{
|
{
|
||||||
throw Exception("Query is executing too slow: " + toString(total_rows / total_elapsed)
|
double total_elapsed = info.total_stopwatch.elapsedSeconds();
|
||||||
+ " rows/sec., minimum: " + toString(limits.min_execution_speed),
|
|
||||||
ErrorCodes::TOO_SLOW);
|
if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0
|
||||||
|
&& total_rows / total_elapsed < limits.min_execution_speed)
|
||||||
|
{
|
||||||
|
throw Exception("Query is executing too slow: " + toString(total_rows / total_elapsed)
|
||||||
|
+ " rows/sec., minimum: " + toString(limits.min_execution_speed),
|
||||||
|
ErrorCodes::TOO_SLOW);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (quota != nullptr && limits.mode == LIMITS_TOTAL)
|
if (quota != nullptr && limits.mode == LIMITS_TOTAL)
|
||||||
@ -405,7 +351,7 @@ void IProfilingBlockInputStream::progressImpl(size_t rows, size_t bytes)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
const BlockStreamProfileInfo & IProfilingBlockInputStream::getInfo() const
|
const BlockStreamProfileInfo & IProfilingBlockInputStream::getInfo() const
|
||||||
{
|
{
|
||||||
@ -427,7 +373,7 @@ void IProfilingBlockInputStream::cancel()
|
|||||||
void IProfilingBlockInputStream::setProgressCallback(ProgressCallback callback)
|
void IProfilingBlockInputStream::setProgressCallback(ProgressCallback callback)
|
||||||
{
|
{
|
||||||
progress_callback = callback;
|
progress_callback = callback;
|
||||||
|
|
||||||
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
|
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
|
||||||
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&**it))
|
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&**it))
|
||||||
child->setProgressCallback(callback);
|
child->setProgressCallback(callback);
|
||||||
|
@ -18,7 +18,7 @@ Block MergeSortingBlockInputStream::readImpl()
|
|||||||
|
|
||||||
if (has_been_read)
|
if (has_been_read)
|
||||||
return Block();
|
return Block();
|
||||||
|
|
||||||
has_been_read = true;
|
has_been_read = true;
|
||||||
|
|
||||||
Blocks blocks;
|
Blocks blocks;
|
||||||
@ -38,15 +38,15 @@ Block MergeSortingBlockInputStream::merge(Blocks & blocks)
|
|||||||
|
|
||||||
if (blocks.size() == 1)
|
if (blocks.size() == 1)
|
||||||
return blocks[0];
|
return blocks[0];
|
||||||
|
|
||||||
Stopwatch watch;
|
Stopwatch watch;
|
||||||
|
|
||||||
LOG_DEBUG(log, "Merge sorting");
|
LOG_DEBUG(log, "Merge sorting");
|
||||||
|
|
||||||
CursorImpls cursors(blocks.size());
|
CursorImpls cursors(blocks.size());
|
||||||
|
|
||||||
bool has_collation = false;
|
bool has_collation = false;
|
||||||
|
|
||||||
size_t i = 0;
|
size_t i = 0;
|
||||||
for (Blocks::const_iterator it = blocks.begin(); it != blocks.end(); ++it, ++i)
|
for (Blocks::const_iterator it = blocks.begin(); it != blocks.end(); ++it, ++i)
|
||||||
{
|
{
|
||||||
@ -56,20 +56,22 @@ Block MergeSortingBlockInputStream::merge(Blocks & blocks)
|
|||||||
cursors[i] = SortCursorImpl(*it, description);
|
cursors[i] = SortCursorImpl(*it, description);
|
||||||
has_collation |= cursors[i].has_collation;
|
has_collation |= cursors[i].has_collation;
|
||||||
}
|
}
|
||||||
|
|
||||||
Block merged;
|
Block merged;
|
||||||
|
|
||||||
if (has_collation)
|
if (has_collation)
|
||||||
merged = mergeImpl<SortCursorWithCollation>(blocks, cursors);
|
merged = mergeImpl<SortCursorWithCollation>(blocks, cursors);
|
||||||
else
|
else
|
||||||
merged = mergeImpl<SortCursor>(blocks, cursors);
|
merged = mergeImpl<SortCursor>(blocks, cursors);
|
||||||
|
|
||||||
|
watch.stop();
|
||||||
|
|
||||||
LOG_DEBUG(log, std::fixed << std::setprecision(2)
|
LOG_DEBUG(log, std::fixed << std::setprecision(2)
|
||||||
<< "Merge sorted " << blocks.size() << " blocks, " << merged.rows() << " rows"
|
<< "Merge sorted " << blocks.size() << " blocks, " << merged.rows() << " rows"
|
||||||
<< " in " << watch.elapsedSeconds() << " sec., "
|
<< " in " << watch.elapsedSeconds() << " sec., "
|
||||||
<< merged.rows() / watch.elapsedSeconds() << " rows/sec., "
|
<< merged.rows() / watch.elapsedSeconds() << " rows/sec., "
|
||||||
<< merged.bytes() / 1000000.0 / watch.elapsedSeconds() << " MiB/sec.");
|
<< merged.bytes() / 1000000.0 / watch.elapsedSeconds() << " MiB/sec.");
|
||||||
|
|
||||||
return merged;
|
return merged;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -78,13 +80,13 @@ Block MergeSortingBlockInputStream::mergeImpl(Blocks & blocks, CursorImpls & cur
|
|||||||
{
|
{
|
||||||
Block merged = blocks[0].cloneEmpty();
|
Block merged = blocks[0].cloneEmpty();
|
||||||
size_t num_columns = blocks[0].columns();
|
size_t num_columns = blocks[0].columns();
|
||||||
|
|
||||||
typedef std::priority_queue<TSortCursor> Queue;
|
typedef std::priority_queue<TSortCursor> Queue;
|
||||||
Queue queue;
|
Queue queue;
|
||||||
|
|
||||||
for (size_t i = 0; i < cursors.size(); ++i)
|
for (size_t i = 0; i < cursors.size(); ++i)
|
||||||
queue.push(TSortCursor(&cursors[i]));
|
queue.push(TSortCursor(&cursors[i]));
|
||||||
|
|
||||||
ColumnPlainPtrs merged_columns;
|
ColumnPlainPtrs merged_columns;
|
||||||
for (size_t i = 0; i < num_columns; ++i) /// TODO: reserve
|
for (size_t i = 0; i < num_columns; ++i) /// TODO: reserve
|
||||||
merged_columns.push_back(&*merged.getByPosition(i).column);
|
merged_columns.push_back(&*merged.getByPosition(i).column);
|
||||||
|
@ -184,7 +184,7 @@ void MergingSortedBlockInputStream::fetchNextBlock(const TSortCursor & current,
|
|||||||
void MergingSortedBlockInputStream::readSuffixImpl()
|
void MergingSortedBlockInputStream::readSuffixImpl()
|
||||||
{
|
{
|
||||||
const BlockStreamProfileInfo & profile_info = getInfo();
|
const BlockStreamProfileInfo & profile_info = getInfo();
|
||||||
double seconds = profile_info.work_stopwatch.elapsedSeconds();
|
double seconds = profile_info.total_stopwatch.elapsedSeconds();
|
||||||
LOG_DEBUG(log, std::fixed << std::setprecision(2)
|
LOG_DEBUG(log, std::fixed << std::setprecision(2)
|
||||||
<< "Merge sorted " << profile_info.blocks << " blocks, " << profile_info.rows << " rows"
|
<< "Merge sorted " << profile_info.blocks << " blocks, " << profile_info.rows << " rows"
|
||||||
<< " in " << seconds << " sec., "
|
<< " in " << seconds << " sec., "
|
||||||
|
@ -1,252 +1,66 @@
|
|||||||
#include <DB/Functions/FunctionsArithmetic.h>
|
|
||||||
#include <DB/Functions/FunctionsComparison.h>
|
|
||||||
#include <DB/Functions/FunctionsLogical.h>
|
|
||||||
#include <DB/Functions/FunctionsString.h>
|
|
||||||
#include <DB/Functions/FunctionsConversion.h>
|
|
||||||
#include <DB/Functions/FunctionsDateTime.h>
|
|
||||||
#include <DB/Functions/FunctionsStringSearch.h>
|
|
||||||
#include <DB/Functions/FunctionsHashing.h>
|
|
||||||
#include <DB/Functions/FunctionsRandom.h>
|
|
||||||
#include <DB/Functions/FunctionsURL.h>
|
|
||||||
#include <DB/Functions/FunctionsArray.h>
|
|
||||||
#include <DB/Functions/FunctionsStringArray.h>
|
|
||||||
#include <DB/Functions/FunctionsConditional.h>
|
|
||||||
#include <DB/Functions/FunctionsDictionaries.h>
|
|
||||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
|
||||||
#include <DB/Functions/FunctionsRound.h>
|
|
||||||
#include <DB/Functions/FunctionsReinterpret.h>
|
|
||||||
#include <DB/Functions/FunctionsFormatting.h>
|
|
||||||
#include <DB/Functions/FunctionsCoding.h>
|
|
||||||
#include <DB/Functions/FunctionsHigherOrder.h>
|
|
||||||
#include <DB/Functions/FunctionsVisitParam.h>
|
|
||||||
|
|
||||||
#include <DB/Functions/FunctionFactory.h>
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
/** Эти функции определены в отдельных translation unit-ах.
|
||||||
|
* Это сделано для того, чтобы уменьшить потребление оперативки при сборке, и ускорить параллельную сборку.
|
||||||
|
*/
|
||||||
|
void registerFunctionsArithmetic(FunctionFactory &);
|
||||||
|
void registerFunctionsArray(FunctionFactory &);
|
||||||
|
void registerFunctionsCoding(FunctionFactory &);
|
||||||
|
void registerFunctionsComparison(FunctionFactory &);
|
||||||
|
void registerFunctionsConditional(FunctionFactory &);
|
||||||
|
void registerFunctionsConversion(FunctionFactory &);
|
||||||
|
void registerFunctionsDateTime(FunctionFactory &);
|
||||||
|
void registerFunctionsDictionaries(FunctionFactory &);
|
||||||
|
void registerFunctionsFormatting(FunctionFactory &);
|
||||||
|
void registerFunctionsHashing(FunctionFactory &);
|
||||||
|
void registerFunctionsHigherOrder(FunctionFactory &);
|
||||||
|
void registerFunctionsLogical(FunctionFactory &);
|
||||||
|
void registerFunctionsMiscellaneous(FunctionFactory &);
|
||||||
|
void registerFunctionsRandom(FunctionFactory &);
|
||||||
|
void registerFunctionsReinterpret(FunctionFactory &);
|
||||||
|
void registerFunctionsRound(FunctionFactory &);
|
||||||
|
void registerFunctionsString(FunctionFactory &);
|
||||||
|
void registerFunctionsStringArray(FunctionFactory &);
|
||||||
|
void registerFunctionsStringSearch(FunctionFactory &);
|
||||||
|
void registerFunctionsURL(FunctionFactory &);
|
||||||
|
void registerFunctionsVisitParam(FunctionFactory &);
|
||||||
|
|
||||||
|
|
||||||
|
FunctionFactory::FunctionFactory()
|
||||||
|
{
|
||||||
|
registerFunctionsArithmetic(*this);
|
||||||
|
registerFunctionsArray(*this);
|
||||||
|
registerFunctionsCoding(*this);
|
||||||
|
registerFunctionsComparison(*this);
|
||||||
|
registerFunctionsConditional(*this);
|
||||||
|
registerFunctionsConversion(*this);
|
||||||
|
registerFunctionsDateTime(*this);
|
||||||
|
registerFunctionsDictionaries(*this);
|
||||||
|
registerFunctionsFormatting(*this);
|
||||||
|
registerFunctionsHashing(*this);
|
||||||
|
registerFunctionsHigherOrder(*this);
|
||||||
|
registerFunctionsLogical(*this);
|
||||||
|
registerFunctionsMiscellaneous(*this);
|
||||||
|
registerFunctionsRandom(*this);
|
||||||
|
registerFunctionsReinterpret(*this);
|
||||||
|
registerFunctionsRound(*this);
|
||||||
|
registerFunctionsString(*this);
|
||||||
|
registerFunctionsStringArray(*this);
|
||||||
|
registerFunctionsStringSearch(*this);
|
||||||
|
registerFunctionsURL(*this);
|
||||||
|
registerFunctionsVisitParam(*this);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
FunctionPtr FunctionFactory::get(
|
FunctionPtr FunctionFactory::get(
|
||||||
const String & name,
|
const String & name,
|
||||||
const Context & context) const
|
const Context & context) const
|
||||||
{
|
{
|
||||||
static const std::unordered_map<
|
|
||||||
std::string,
|
|
||||||
std::function<IFunction* (const Context & context)>> functions =
|
|
||||||
{
|
|
||||||
#define F [](const Context & context)
|
|
||||||
{"plus", F { return new FunctionPlus; } },
|
|
||||||
{"minus", F { return new FunctionMinus; } },
|
|
||||||
{"multiply", F { return new FunctionMultiply; } },
|
|
||||||
{"divide", F { return new FunctionDivideFloating; } },
|
|
||||||
{"intDiv", F { return new FunctionDivideIntegral; } },
|
|
||||||
{"modulo", F { return new FunctionModulo; } },
|
|
||||||
{"negate", F { return new FunctionNegate; } },
|
|
||||||
{"bitAnd", F { return new FunctionBitAnd; } },
|
|
||||||
{"bitOr", F { return new FunctionBitOr; } },
|
|
||||||
{"bitXor", F { return new FunctionBitXor; } },
|
|
||||||
{"bitNot", F { return new FunctionBitNot; } },
|
|
||||||
{"bitShiftLeft", F { return new FunctionBitShiftLeft; } },
|
|
||||||
{"bitShiftRight", F { return new FunctionBitShiftRight; } },
|
|
||||||
|
|
||||||
{"equals", F { return new FunctionEquals; } },
|
|
||||||
{"notEquals", F { return new FunctionNotEquals; } },
|
|
||||||
{"less", F { return new FunctionLess; } },
|
|
||||||
{"greater", F { return new FunctionGreater; } },
|
|
||||||
{"lessOrEquals", F { return new FunctionLessOrEquals; } },
|
|
||||||
{"greaterOrEquals", F { return new FunctionGreaterOrEquals; } },
|
|
||||||
|
|
||||||
{"and", F { return new FunctionAnd; } },
|
|
||||||
{"or", F { return new FunctionOr; } },
|
|
||||||
{"xor", F { return new FunctionXor; } },
|
|
||||||
{"not", F { return new FunctionNot; } },
|
|
||||||
|
|
||||||
{"roundToExp2", F { return new FunctionRoundToExp2; } },
|
|
||||||
{"roundDuration", F { return new FunctionRoundDuration; } },
|
|
||||||
{"roundAge", F { return new FunctionRoundAge; } },
|
|
||||||
|
|
||||||
{"empty", F { return new FunctionEmpty; } },
|
|
||||||
{"notEmpty", F { return new FunctionNotEmpty; } },
|
|
||||||
{"length", F { return new FunctionLength; } },
|
|
||||||
{"lengthUTF8", F { return new FunctionLengthUTF8; } },
|
|
||||||
{"lower", F { return new FunctionLower; } },
|
|
||||||
{"upper", F { return new FunctionUpper; } },
|
|
||||||
{"lowerUTF8", F { return new FunctionLowerUTF8; } },
|
|
||||||
{"upperUTF8", F { return new FunctionUpperUTF8; } },
|
|
||||||
{"reverse", F { return new FunctionReverse; } },
|
|
||||||
{"reverseUTF8", F { return new FunctionReverseUTF8; } },
|
|
||||||
{"concat", F { return new FunctionConcat; } },
|
|
||||||
{"substring", F { return new FunctionSubstring; } },
|
|
||||||
{"replaceOne", F { return new FunctionReplaceOne; } },
|
|
||||||
{"replaceAll", F { return new FunctionReplaceAll; } },
|
|
||||||
{"replaceRegexpOne", F { return new FunctionReplaceRegexpOne; } },
|
|
||||||
{"replaceRegexpAll", F { return new FunctionReplaceRegexpAll; } },
|
|
||||||
{"substringUTF8", F { return new FunctionSubstringUTF8; } },
|
|
||||||
|
|
||||||
{"toUInt8", F { return new FunctionToUInt8; } },
|
|
||||||
{"toUInt16", F { return new FunctionToUInt16; } },
|
|
||||||
{"toUInt32", F { return new FunctionToUInt32; } },
|
|
||||||
{"toUInt64", F { return new FunctionToUInt64; } },
|
|
||||||
{"toInt8", F { return new FunctionToInt8; } },
|
|
||||||
{"toInt16", F { return new FunctionToInt16; } },
|
|
||||||
{"toInt32", F { return new FunctionToInt32; } },
|
|
||||||
{"toInt64", F { return new FunctionToInt64; } },
|
|
||||||
{"toFloat32", F { return new FunctionToFloat32; } },
|
|
||||||
{"toFloat64", F { return new FunctionToFloat64; } },
|
|
||||||
{"toDate", F { return new FunctionToDate; } },
|
|
||||||
{"toDateTime", F { return new FunctionToDateTime; } },
|
|
||||||
{"toString", F { return new FunctionToString; } },
|
|
||||||
{"toFixedString", F { return new FunctionToFixedString; } },
|
|
||||||
{"toStringCutToZero", F { return new FunctionToStringCutToZero; } },
|
|
||||||
|
|
||||||
{"reinterpretAsUInt8", F { return new FunctionReinterpretAsUInt8; } },
|
|
||||||
{"reinterpretAsUInt16", F { return new FunctionReinterpretAsUInt16; } },
|
|
||||||
{"reinterpretAsUInt32", F { return new FunctionReinterpretAsUInt32; } },
|
|
||||||
{"reinterpretAsUInt64", F { return new FunctionReinterpretAsUInt64; } },
|
|
||||||
{"reinterpretAsInt8", F { return new FunctionReinterpretAsInt8; } },
|
|
||||||
{"reinterpretAsInt16", F { return new FunctionReinterpretAsInt16; } },
|
|
||||||
{"reinterpretAsInt32", F { return new FunctionReinterpretAsInt32; } },
|
|
||||||
{"reinterpretAsInt64", F { return new FunctionReinterpretAsInt64; } },
|
|
||||||
{"reinterpretAsFloat32", F { return new FunctionReinterpretAsFloat32; } },
|
|
||||||
{"reinterpretAsFloat64", F { return new FunctionReinterpretAsFloat64; } },
|
|
||||||
{"reinterpretAsDate", F { return new FunctionReinterpretAsDate; } },
|
|
||||||
{"reinterpretAsDateTime", F { return new FunctionReinterpretAsDateTime; } },
|
|
||||||
{"reinterpretAsString", F { return new FunctionReinterpretAsString; } },
|
|
||||||
|
|
||||||
{"toYear", F { return new FunctionToYear; } },
|
|
||||||
{"toMonth", F { return new FunctionToMonth; } },
|
|
||||||
{"toDayOfMonth", F { return new FunctionToDayOfMonth; } },
|
|
||||||
{"toDayOfWeek", F { return new FunctionToDayOfWeek; } },
|
|
||||||
{"toHour", F { return new FunctionToHour; } },
|
|
||||||
{"toMinute", F { return new FunctionToMinute; } },
|
|
||||||
{"toSecond", F { return new FunctionToSecond; } },
|
|
||||||
{"toMonday", F { return new FunctionToMonday; } },
|
|
||||||
{"toStartOfMonth", F { return new FunctionToStartOfMonth; } },
|
|
||||||
{"toStartOfQuarter", F { return new FunctionToStartOfQuarter; } },
|
|
||||||
{"toStartOfYear", F { return new FunctionToStartOfYear; } },
|
|
||||||
{"toStartOfMinute", F { return new FunctionToStartOfMinute; } },
|
|
||||||
{"toStartOfHour", F { return new FunctionToStartOfHour; } },
|
|
||||||
{"toRelativeYearNum", F { return new FunctionToRelativeYearNum; } },
|
|
||||||
{"toRelativeMonthNum", F { return new FunctionToRelativeMonthNum; } },
|
|
||||||
{"toRelativeWeekNum", F { return new FunctionToRelativeWeekNum; } },
|
|
||||||
{"toRelativeDayNum", F { return new FunctionToRelativeDayNum; } },
|
|
||||||
{"toRelativeHourNum", F { return new FunctionToRelativeHourNum; } },
|
|
||||||
{"toRelativeMinuteNum", F { return new FunctionToRelativeMinuteNum; } },
|
|
||||||
{"toRelativeSecondNum", F { return new FunctionToRelativeSecondNum; } },
|
|
||||||
{"toTime", F { return new FunctionToTime; } },
|
|
||||||
{"now", F { return new FunctionNow; } },
|
|
||||||
{"timeSlot", F { return new FunctionTimeSlot; } },
|
|
||||||
{"timeSlots", F { return new FunctionTimeSlots; } },
|
|
||||||
|
|
||||||
{"position", F { return new FunctionPosition; } },
|
|
||||||
{"positionUTF8", F { return new FunctionPositionUTF8; } },
|
|
||||||
{"match", F { return new FunctionMatch; } },
|
|
||||||
{"like", F { return new FunctionLike; } },
|
|
||||||
{"notLike", F { return new FunctionNotLike; } },
|
|
||||||
{"extract", F { return new FunctionExtract; } },
|
|
||||||
{"extractAll", F { return new FunctionExtractAll; } },
|
|
||||||
|
|
||||||
{"halfMD5", F { return new FunctionHalfMD5; } },
|
|
||||||
{"sipHash64", F { return new FunctionSipHash64; } },
|
|
||||||
{"cityHash64", F { return new FunctionCityHash64; } },
|
|
||||||
{"intHash32", F { return new FunctionIntHash32; } },
|
|
||||||
{"intHash64", F { return new FunctionIntHash64; } },
|
|
||||||
|
|
||||||
{"IPv4NumToString", F { return new FunctionIPv4NumToString; } },
|
|
||||||
{"IPv4StringToNum", F { return new FunctionIPv4StringToNum; } },
|
|
||||||
{"hex", F { return new FunctionHex; } },
|
|
||||||
{"unhex", F { return new FunctionUnhex; } },
|
|
||||||
{"bitmaskToList", F { return new FunctionBitmaskToList; } },
|
|
||||||
{"bitmaskToArray", F { return new FunctionBitmaskToArray; } },
|
|
||||||
|
|
||||||
{"rand", F { return new FunctionRand; } },
|
|
||||||
{"rand64", F { return new FunctionRand64; } },
|
|
||||||
|
|
||||||
{"protocol", F { return new FunctionProtocol; } },
|
|
||||||
{"domain", F { return new FunctionDomain; } },
|
|
||||||
{"domainWithoutWWW", F { return new FunctionDomainWithoutWWW; } },
|
|
||||||
{"topLevelDomain", F { return new FunctionTopLevelDomain; } },
|
|
||||||
{"path", F { return new FunctionPath; } },
|
|
||||||
{"queryString", F { return new FunctionQueryString; } },
|
|
||||||
{"fragment", F { return new FunctionFragment; } },
|
|
||||||
{"queryStringAndFragment", F { return new FunctionQueryStringAndFragment; } },
|
|
||||||
{"extractURLParameter", F { return new FunctionExtractURLParameter; } },
|
|
||||||
{"extractURLParameters", F { return new FunctionExtractURLParameters; } },
|
|
||||||
{"extractURLParameterNames", F { return new FunctionExtractURLParameterNames; } },
|
|
||||||
{"URLHierarchy", F { return new FunctionURLHierarchy; } },
|
|
||||||
{"URLPathHierarchy", F { return new FunctionURLPathHierarchy; } },
|
|
||||||
{"cutWWW", F { return new FunctionCutWWW; } },
|
|
||||||
{"cutQueryString", F { return new FunctionCutQueryString; } },
|
|
||||||
{"cutFragment", F { return new FunctionCutFragment; } },
|
|
||||||
{"cutQueryStringAndFragment", F { return new FunctionCutQueryStringAndFragment; } },
|
|
||||||
{"cutURLParameter", F { return new FunctionCutURLParameter; } },
|
|
||||||
|
|
||||||
{"hostName", F { return new FunctionHostName; } },
|
|
||||||
{"visibleWidth", F { return new FunctionVisibleWidth; } },
|
|
||||||
{"bar", F { return new FunctionBar; } },
|
|
||||||
{"toTypeName", F { return new FunctionToTypeName; } },
|
|
||||||
{"blockSize", F { return new FunctionBlockSize; } },
|
|
||||||
{"sleep", F { return new FunctionSleep; } },
|
|
||||||
{"materialize", F { return new FunctionMaterialize; } },
|
|
||||||
{"ignore", F { return new FunctionIgnore; } },
|
|
||||||
{"arrayJoin", F { return new FunctionArrayJoin; } },
|
|
||||||
|
|
||||||
{"tuple", F { return new FunctionTuple; } },
|
|
||||||
{"tupleElement", F { return new FunctionTupleElement; } },
|
|
||||||
{"in", F { return new FunctionIn(false, false); } },
|
|
||||||
{"notIn", F { return new FunctionIn(true, false); } },
|
|
||||||
{"globalIn", F { return new FunctionIn(false, true); } },
|
|
||||||
{"globalNotIn", F { return new FunctionIn(true, true); } },
|
|
||||||
|
|
||||||
{"array", F { return new FunctionArray; } },
|
|
||||||
{"arrayElement", F { return new FunctionArrayElement; } },
|
|
||||||
{"has", F { return new FunctionHas; } },
|
|
||||||
{"indexOf", F { return new FunctionIndexOf; } },
|
|
||||||
{"countEqual", F { return new FunctionCountEqual; } },
|
|
||||||
{"arrayEnumerate", F { return new FunctionArrayEnumerate; } },
|
|
||||||
{"arrayEnumerateUniq", F { return new FunctionArrayEnumerateUniq; } },
|
|
||||||
|
|
||||||
{"arrayMap", F { return new FunctionArrayMap; } },
|
|
||||||
{"arrayFilter", F { return new FunctionArrayFilter; } },
|
|
||||||
{"arrayCount", F { return new FunctionArrayCount; } },
|
|
||||||
{"arrayExists", F { return new FunctionArrayExists; } },
|
|
||||||
{"arrayAll", F { return new FunctionArrayAll; } },
|
|
||||||
{"arraySum", F { return new FunctionArraySum; } },
|
|
||||||
|
|
||||||
{"alphaTokens", F { return new FunctionAlphaTokens; } },
|
|
||||||
{"splitByChar", F { return new FunctionSplitByChar; } },
|
|
||||||
{"splitByString", F { return new FunctionSplitByString; } },
|
|
||||||
|
|
||||||
{"if", F { return new FunctionIf; } },
|
|
||||||
|
|
||||||
{"regionToCity", F { return new FunctionRegionToCity(context.getDictionaries().getRegionsHierarchies()); } },
|
|
||||||
{"regionToArea", F { return new FunctionRegionToArea(context.getDictionaries().getRegionsHierarchies()); } },
|
|
||||||
{"regionToCountry", F { return new FunctionRegionToCountry(context.getDictionaries().getRegionsHierarchies()); } },
|
|
||||||
{"regionToContinent", F { return new FunctionRegionToContinent(context.getDictionaries().getRegionsHierarchies()); } },
|
|
||||||
{"OSToRoot", F { return new FunctionOSToRoot(context.getDictionaries().getTechDataHierarchy()); } },
|
|
||||||
{"SEToRoot", F { return new FunctionSEToRoot(context.getDictionaries().getTechDataHierarchy()); } },
|
|
||||||
{"categoryToRoot", F { return new FunctionCategoryToRoot(context.getDictionaries().getCategoriesHierarchy()); } },
|
|
||||||
{"categoryToSecondLevel", F { return new FunctionCategoryToSecondLevel(context.getDictionaries().getCategoriesHierarchy()); } },
|
|
||||||
{"regionIn", F { return new FunctionRegionIn(context.getDictionaries().getRegionsHierarchies()); } },
|
|
||||||
{"OSIn", F { return new FunctionOSIn(context.getDictionaries().getTechDataHierarchy()); } },
|
|
||||||
{"SEIn", F { return new FunctionSEIn(context.getDictionaries().getTechDataHierarchy()); } },
|
|
||||||
{"categoryIn", F { return new FunctionCategoryIn(context.getDictionaries().getCategoriesHierarchy()); } },
|
|
||||||
{"regionHierarchy", F { return new FunctionRegionHierarchy(context.getDictionaries().getRegionsHierarchies()); } },
|
|
||||||
{"OSHierarchy", F { return new FunctionOSHierarchy(context.getDictionaries().getTechDataHierarchy()); } },
|
|
||||||
{"SEHierarchy", F { return new FunctionSEHierarchy(context.getDictionaries().getTechDataHierarchy()); } },
|
|
||||||
{"categoryHierarchy", F { return new FunctionCategoryHierarchy(context.getDictionaries().getCategoriesHierarchy()); } },
|
|
||||||
{"regionToName", F { return new FunctionRegionToName(context.getDictionaries().getRegionsNames()); } },
|
|
||||||
|
|
||||||
{"visitParamHas", F { return new FunctionVisitParamHas; } },
|
|
||||||
{"visitParamExtractUInt", F { return new FunctionVisitParamExtractUInt; } },
|
|
||||||
{"visitParamExtractInt", F { return new FunctionVisitParamExtractInt; } },
|
|
||||||
{"visitParamExtractFloat", F { return new FunctionVisitParamExtractFloat; } },
|
|
||||||
{"visitParamExtractBool", F { return new FunctionVisitParamExtractBool; } },
|
|
||||||
{"visitParamExtractRaw", F { return new FunctionVisitParamExtractRaw; } },
|
|
||||||
{"visitParamExtractString", F { return new FunctionVisitParamExtractString; } },
|
|
||||||
};
|
|
||||||
|
|
||||||
auto it = functions.find(name);
|
auto it = functions.find(name);
|
||||||
if (functions.end() != it)
|
if (functions.end() != it)
|
||||||
return it->second(context);
|
return it->second(context);
|
||||||
|
28
dbms/src/Functions/FunctionsArithmetic.cpp
Normal file
28
dbms/src/Functions/FunctionsArithmetic.cpp
Normal file
@ -0,0 +1,28 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsArithmetic.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsArithmetic(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("plus", F { return new FunctionPlus; });
|
||||||
|
factory.registerFunction("minus", F { return new FunctionMinus; });
|
||||||
|
factory.registerFunction("multiply", F { return new FunctionMultiply; });
|
||||||
|
factory.registerFunction("divide", F { return new FunctionDivideFloating; });
|
||||||
|
factory.registerFunction("intDiv", F { return new FunctionDivideIntegral; });
|
||||||
|
factory.registerFunction("modulo", F { return new FunctionModulo; });
|
||||||
|
factory.registerFunction("negate", F { return new FunctionNegate; });
|
||||||
|
factory.registerFunction("bitAnd", F { return new FunctionBitAnd; });
|
||||||
|
factory.registerFunction("bitOr", F { return new FunctionBitOr; });
|
||||||
|
factory.registerFunction("bitXor", F { return new FunctionBitXor; });
|
||||||
|
factory.registerFunction("bitNot", F { return new FunctionBitNot; });
|
||||||
|
factory.registerFunction("bitShiftLeft", F { return new FunctionBitShiftLeft; });
|
||||||
|
factory.registerFunction("bitShiftRight", F { return new FunctionBitShiftRight; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
22
dbms/src/Functions/FunctionsArray.cpp
Normal file
22
dbms/src/Functions/FunctionsArray.cpp
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsArray.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsArray(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("array", F { return new FunctionArray; });
|
||||||
|
factory.registerFunction("arrayElement", F { return new FunctionArrayElement; });
|
||||||
|
factory.registerFunction("has", F { return new FunctionHas; });
|
||||||
|
factory.registerFunction("indexOf", F { return new FunctionIndexOf; });
|
||||||
|
factory.registerFunction("countEqual", F { return new FunctionCountEqual; });
|
||||||
|
factory.registerFunction("arrayEnumerate", F { return new FunctionArrayEnumerate; });
|
||||||
|
factory.registerFunction("arrayEnumerateUniq", F { return new FunctionArrayEnumerateUniq; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
21
dbms/src/Functions/FunctionsCoding.cpp
Normal file
21
dbms/src/Functions/FunctionsCoding.cpp
Normal file
@ -0,0 +1,21 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsCoding.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsCoding(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("toStringCutToZero", F { return new FunctionToStringCutToZero; });
|
||||||
|
factory.registerFunction("IPv4NumToString", F { return new FunctionIPv4NumToString; });
|
||||||
|
factory.registerFunction("IPv4StringToNum", F { return new FunctionIPv4StringToNum; });
|
||||||
|
factory.registerFunction("hex", F { return new FunctionHex; });
|
||||||
|
factory.registerFunction("unhex", F { return new FunctionUnhex; });
|
||||||
|
factory.registerFunction("bitmaskToArray", F { return new FunctionBitmaskToArray; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
21
dbms/src/Functions/FunctionsComparison.cpp
Normal file
21
dbms/src/Functions/FunctionsComparison.cpp
Normal file
@ -0,0 +1,21 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsComparison.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsComparison(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("equals", F { return new FunctionEquals; });
|
||||||
|
factory.registerFunction("notEquals", F { return new FunctionNotEquals; });
|
||||||
|
factory.registerFunction("less", F { return new FunctionLess; });
|
||||||
|
factory.registerFunction("greater", F { return new FunctionGreater; });
|
||||||
|
factory.registerFunction("lessOrEquals", F { return new FunctionLessOrEquals; });
|
||||||
|
factory.registerFunction("greaterOrEquals", F { return new FunctionGreaterOrEquals; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
16
dbms/src/Functions/FunctionsConditional.cpp
Normal file
16
dbms/src/Functions/FunctionsConditional.cpp
Normal file
@ -0,0 +1,16 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsConditional.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsConditional(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("if", F { return new FunctionIf; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
29
dbms/src/Functions/FunctionsConversion.cpp
Normal file
29
dbms/src/Functions/FunctionsConversion.cpp
Normal file
@ -0,0 +1,29 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsConversion.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsConversion(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("toUInt8", F { return new FunctionToUInt8; });
|
||||||
|
factory.registerFunction("toUInt16", F { return new FunctionToUInt16; });
|
||||||
|
factory.registerFunction("toUInt32", F { return new FunctionToUInt32; });
|
||||||
|
factory.registerFunction("toUInt64", F { return new FunctionToUInt64; });
|
||||||
|
factory.registerFunction("toInt8", F { return new FunctionToInt8; });
|
||||||
|
factory.registerFunction("toInt16", F { return new FunctionToInt16; });
|
||||||
|
factory.registerFunction("toInt32", F { return new FunctionToInt32; });
|
||||||
|
factory.registerFunction("toInt64", F { return new FunctionToInt64; });
|
||||||
|
factory.registerFunction("toFloat32", F { return new FunctionToFloat32; });
|
||||||
|
factory.registerFunction("toFloat64", F { return new FunctionToFloat64; });
|
||||||
|
factory.registerFunction("toDate", F { return new FunctionToDate; });
|
||||||
|
factory.registerFunction("toDateTime", F { return new FunctionToDateTime; });
|
||||||
|
factory.registerFunction("toString", F { return new FunctionToString; });
|
||||||
|
factory.registerFunction("toFixedString", F { return new FunctionToFixedString; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
39
dbms/src/Functions/FunctionsDateTime.cpp
Normal file
39
dbms/src/Functions/FunctionsDateTime.cpp
Normal file
@ -0,0 +1,39 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsDateTime.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsDateTime(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("toYear", F { return new FunctionToYear; });
|
||||||
|
factory.registerFunction("toMonth", F { return new FunctionToMonth; });
|
||||||
|
factory.registerFunction("toDayOfMonth", F { return new FunctionToDayOfMonth; });
|
||||||
|
factory.registerFunction("toDayOfWeek", F { return new FunctionToDayOfWeek; });
|
||||||
|
factory.registerFunction("toHour", F { return new FunctionToHour; });
|
||||||
|
factory.registerFunction("toMinute", F { return new FunctionToMinute; });
|
||||||
|
factory.registerFunction("toSecond", F { return new FunctionToSecond; });
|
||||||
|
factory.registerFunction("toMonday", F { return new FunctionToMonday; });
|
||||||
|
factory.registerFunction("toStartOfMonth", F { return new FunctionToStartOfMonth; });
|
||||||
|
factory.registerFunction("toStartOfQuarter", F { return new FunctionToStartOfQuarter; });
|
||||||
|
factory.registerFunction("toStartOfYear", F { return new FunctionToStartOfYear; });
|
||||||
|
factory.registerFunction("toStartOfMinute", F { return new FunctionToStartOfMinute; });
|
||||||
|
factory.registerFunction("toStartOfHour", F { return new FunctionToStartOfHour; });
|
||||||
|
factory.registerFunction("toRelativeYearNum", F { return new FunctionToRelativeYearNum; });
|
||||||
|
factory.registerFunction("toRelativeMonthNum", F { return new FunctionToRelativeMonthNum; });
|
||||||
|
factory.registerFunction("toRelativeWeekNum", F { return new FunctionToRelativeWeekNum; });
|
||||||
|
factory.registerFunction("toRelativeDayNum", F { return new FunctionToRelativeDayNum; });
|
||||||
|
factory.registerFunction("toRelativeHourNum", F { return new FunctionToRelativeHourNum; });
|
||||||
|
factory.registerFunction("toRelativeMinuteNum", F { return new FunctionToRelativeMinuteNum; });
|
||||||
|
factory.registerFunction("toRelativeSecondNum", F { return new FunctionToRelativeSecondNum; });
|
||||||
|
factory.registerFunction("toTime", F { return new FunctionToTime; });
|
||||||
|
factory.registerFunction("now", F { return new FunctionNow; });
|
||||||
|
factory.registerFunction("timeSlot", F { return new FunctionTimeSlot; });
|
||||||
|
factory.registerFunction("timeSlots", F { return new FunctionTimeSlots; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
49
dbms/src/Functions/FunctionsDictionaries.cpp
Normal file
49
dbms/src/Functions/FunctionsDictionaries.cpp
Normal file
@ -0,0 +1,49 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsDictionaries.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsDictionaries(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("regionToCity",
|
||||||
|
F { return new FunctionRegionToCity(context.getDictionaries().getRegionsHierarchies()); });
|
||||||
|
factory.registerFunction("regionToArea",
|
||||||
|
F { return new FunctionRegionToArea(context.getDictionaries().getRegionsHierarchies()); });
|
||||||
|
factory.registerFunction("regionToCountry",
|
||||||
|
F { return new FunctionRegionToCountry(context.getDictionaries().getRegionsHierarchies()); });
|
||||||
|
factory.registerFunction("regionToContinent",
|
||||||
|
F { return new FunctionRegionToContinent(context.getDictionaries().getRegionsHierarchies()); });
|
||||||
|
factory.registerFunction("OSToRoot",
|
||||||
|
F { return new FunctionOSToRoot(context.getDictionaries().getTechDataHierarchy()); });
|
||||||
|
factory.registerFunction("SEToRoot",
|
||||||
|
F { return new FunctionSEToRoot(context.getDictionaries().getTechDataHierarchy()); });
|
||||||
|
factory.registerFunction("categoryToRoot",
|
||||||
|
F { return new FunctionCategoryToRoot(context.getDictionaries().getCategoriesHierarchy()); });
|
||||||
|
factory.registerFunction("categoryToSecondLevel",
|
||||||
|
F { return new FunctionCategoryToSecondLevel(context.getDictionaries().getCategoriesHierarchy()); });
|
||||||
|
factory.registerFunction("regionIn",
|
||||||
|
F { return new FunctionRegionIn(context.getDictionaries().getRegionsHierarchies()); });
|
||||||
|
factory.registerFunction("OSIn",
|
||||||
|
F { return new FunctionOSIn(context.getDictionaries().getTechDataHierarchy()); });
|
||||||
|
factory.registerFunction("SEIn",
|
||||||
|
F { return new FunctionSEIn(context.getDictionaries().getTechDataHierarchy()); });
|
||||||
|
factory.registerFunction("categoryIn",
|
||||||
|
F { return new FunctionCategoryIn(context.getDictionaries().getCategoriesHierarchy()); });
|
||||||
|
factory.registerFunction("regionHierarchy",
|
||||||
|
F { return new FunctionRegionHierarchy(context.getDictionaries().getRegionsHierarchies()); });
|
||||||
|
factory.registerFunction("OSHierarchy",
|
||||||
|
F { return new FunctionOSHierarchy(context.getDictionaries().getTechDataHierarchy()); });
|
||||||
|
factory.registerFunction("SEHierarchy",
|
||||||
|
F { return new FunctionSEHierarchy(context.getDictionaries().getTechDataHierarchy()); });
|
||||||
|
factory.registerFunction("categoryHierarchy",
|
||||||
|
F { return new FunctionCategoryHierarchy(context.getDictionaries().getCategoriesHierarchy()); });
|
||||||
|
factory.registerFunction("regionToName",
|
||||||
|
F { return new FunctionRegionToName(context.getDictionaries().getRegionsNames()); });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
16
dbms/src/Functions/FunctionsFormatting.cpp
Normal file
16
dbms/src/Functions/FunctionsFormatting.cpp
Normal file
@ -0,0 +1,16 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsFormatting.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsFormatting(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("bitmaskToList", F { return new FunctionBitmaskToList; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
21
dbms/src/Functions/FunctionsHashing.cpp
Normal file
21
dbms/src/Functions/FunctionsHashing.cpp
Normal file
@ -0,0 +1,21 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsHashing.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsHashing(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("halfMD5", F { return new FunctionHalfMD5; });
|
||||||
|
factory.registerFunction("sipHash64", F { return new FunctionSipHash64; });
|
||||||
|
factory.registerFunction("cityHash64", F { return new FunctionCityHash64; });
|
||||||
|
factory.registerFunction("intHash32", F { return new FunctionIntHash32; });
|
||||||
|
factory.registerFunction("intHash64", F { return new FunctionIntHash64; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
21
dbms/src/Functions/FunctionsHigherOrder.cpp
Normal file
21
dbms/src/Functions/FunctionsHigherOrder.cpp
Normal file
@ -0,0 +1,21 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsHigherOrder.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsHigherOrder(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("arrayMap", F { return new FunctionArrayMap; });
|
||||||
|
factory.registerFunction("arrayFilter", F { return new FunctionArrayFilter; });
|
||||||
|
factory.registerFunction("arrayCount", F { return new FunctionArrayCount; });
|
||||||
|
factory.registerFunction("arrayExists", F { return new FunctionArrayExists; });
|
||||||
|
factory.registerFunction("arrayAll", F { return new FunctionArrayAll; });
|
||||||
|
factory.registerFunction("arraySum", F { return new FunctionArraySum; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
19
dbms/src/Functions/FunctionsLogical.cpp
Normal file
19
dbms/src/Functions/FunctionsLogical.cpp
Normal file
@ -0,0 +1,19 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsLogical.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsLogical(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("and", F { return new FunctionAnd; });
|
||||||
|
factory.registerFunction("or", F { return new FunctionOr; });
|
||||||
|
factory.registerFunction("xor", F { return new FunctionXor; });
|
||||||
|
factory.registerFunction("not", F { return new FunctionNot; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -1,5 +1,6 @@
|
|||||||
#include <math.h>
|
#include <math.h>
|
||||||
|
|
||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
#include <DB/Functions/FunctionsArithmetic.h>
|
#include <DB/Functions/FunctionsArithmetic.h>
|
||||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
#include <DB/Functions/FunctionsMiscellaneous.h>
|
||||||
|
|
||||||
@ -297,3 +298,33 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
|
|||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("hostName", F { return new FunctionHostName; });
|
||||||
|
factory.registerFunction("visibleWidth", F { return new FunctionVisibleWidth; });
|
||||||
|
factory.registerFunction("toTypeName", F { return new FunctionToTypeName; });
|
||||||
|
factory.registerFunction("blockSize", F { return new FunctionBlockSize; });
|
||||||
|
factory.registerFunction("sleep", F { return new FunctionSleep; });
|
||||||
|
factory.registerFunction("materialize", F { return new FunctionMaterialize; });
|
||||||
|
factory.registerFunction("ignore", F { return new FunctionIgnore; });
|
||||||
|
factory.registerFunction("arrayJoin", F { return new FunctionArrayJoin; });
|
||||||
|
factory.registerFunction("bar", F { return new FunctionBar; });
|
||||||
|
|
||||||
|
factory.registerFunction("tuple", F { return new FunctionTuple; });
|
||||||
|
factory.registerFunction("tupleElement", F { return new FunctionTupleElement; });
|
||||||
|
factory.registerFunction("in", F { return new FunctionIn(false, false); });
|
||||||
|
factory.registerFunction("notIn", F { return new FunctionIn(true, false); });
|
||||||
|
factory.registerFunction("globalIn", F { return new FunctionIn(false, true); });
|
||||||
|
factory.registerFunction("globalNotIn", F { return new FunctionIn(true, true); });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
17
dbms/src/Functions/FunctionsRandom.cpp
Normal file
17
dbms/src/Functions/FunctionsRandom.cpp
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsRandom.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsRandom(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("rand", F { return new FunctionRand; });
|
||||||
|
factory.registerFunction("rand64", F { return new FunctionRand64; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
28
dbms/src/Functions/FunctionsReinterpret.cpp
Normal file
28
dbms/src/Functions/FunctionsReinterpret.cpp
Normal file
@ -0,0 +1,28 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsReinterpret.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsReinterpret(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("reinterpretAsUInt8", F { return new FunctionReinterpretAsUInt8; });
|
||||||
|
factory.registerFunction("reinterpretAsUInt16", F { return new FunctionReinterpretAsUInt16; });
|
||||||
|
factory.registerFunction("reinterpretAsUInt32", F { return new FunctionReinterpretAsUInt32; });
|
||||||
|
factory.registerFunction("reinterpretAsUInt64", F { return new FunctionReinterpretAsUInt64; });
|
||||||
|
factory.registerFunction("reinterpretAsInt8", F { return new FunctionReinterpretAsInt8; });
|
||||||
|
factory.registerFunction("reinterpretAsInt16", F { return new FunctionReinterpretAsInt16; });
|
||||||
|
factory.registerFunction("reinterpretAsInt32", F { return new FunctionReinterpretAsInt32; });
|
||||||
|
factory.registerFunction("reinterpretAsInt64", F { return new FunctionReinterpretAsInt64; });
|
||||||
|
factory.registerFunction("reinterpretAsFloat32", F { return new FunctionReinterpretAsFloat32; });
|
||||||
|
factory.registerFunction("reinterpretAsFloat64", F { return new FunctionReinterpretAsFloat64; });
|
||||||
|
factory.registerFunction("reinterpretAsDate", F { return new FunctionReinterpretAsDate; });
|
||||||
|
factory.registerFunction("reinterpretAsDateTime", F { return new FunctionReinterpretAsDateTime; });
|
||||||
|
factory.registerFunction("reinterpretAsString", F { return new FunctionReinterpretAsString; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
18
dbms/src/Functions/FunctionsRound.cpp
Normal file
18
dbms/src/Functions/FunctionsRound.cpp
Normal file
@ -0,0 +1,18 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsRound.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsRound(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("roundToExp2", F { return new FunctionRoundToExp2; });
|
||||||
|
factory.registerFunction("roundDuration", F { return new FunctionRoundDuration; });
|
||||||
|
factory.registerFunction("roundAge", F { return new FunctionRoundAge; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
28
dbms/src/Functions/FunctionsString.cpp
Normal file
28
dbms/src/Functions/FunctionsString.cpp
Normal file
@ -0,0 +1,28 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsString.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsString(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("empty", F { return new FunctionEmpty; });
|
||||||
|
factory.registerFunction("notEmpty", F { return new FunctionNotEmpty; });
|
||||||
|
factory.registerFunction("length", F { return new FunctionLength; });
|
||||||
|
factory.registerFunction("lengthUTF8", F { return new FunctionLengthUTF8; });
|
||||||
|
factory.registerFunction("lower", F { return new FunctionLower; });
|
||||||
|
factory.registerFunction("upper", F { return new FunctionUpper; });
|
||||||
|
factory.registerFunction("lowerUTF8", F { return new FunctionLowerUTF8; });
|
||||||
|
factory.registerFunction("upperUTF8", F { return new FunctionUpperUTF8; });
|
||||||
|
factory.registerFunction("reverse", F { return new FunctionReverse; });
|
||||||
|
factory.registerFunction("reverseUTF8", F { return new FunctionReverseUTF8; });
|
||||||
|
factory.registerFunction("concat", F { return new FunctionConcat; });
|
||||||
|
factory.registerFunction("substring", F { return new FunctionSubstring; });
|
||||||
|
factory.registerFunction("substringUTF8", F { return new FunctionSubstringUTF8; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
19
dbms/src/Functions/FunctionsStringArray.cpp
Normal file
19
dbms/src/Functions/FunctionsStringArray.cpp
Normal file
@ -0,0 +1,19 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsStringArray.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsStringArray(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("extractAll", F { return new FunctionExtractAll; });
|
||||||
|
factory.registerFunction("alphaTokens", F { return new FunctionAlphaTokens; });
|
||||||
|
factory.registerFunction("splitByChar", F { return new FunctionSplitByChar; });
|
||||||
|
factory.registerFunction("splitByString", F { return new FunctionSplitByString; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
25
dbms/src/Functions/FunctionsStringSearch.cpp
Normal file
25
dbms/src/Functions/FunctionsStringSearch.cpp
Normal file
@ -0,0 +1,25 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsStringSearch.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsStringSearch(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("replaceOne", F { return new FunctionReplaceOne; });
|
||||||
|
factory.registerFunction("replaceAll", F { return new FunctionReplaceAll; });
|
||||||
|
factory.registerFunction("replaceRegexpOne", F { return new FunctionReplaceRegexpOne; });
|
||||||
|
factory.registerFunction("replaceRegexpAll", F { return new FunctionReplaceRegexpAll; });
|
||||||
|
factory.registerFunction("position", F { return new FunctionPosition; });
|
||||||
|
factory.registerFunction("positionUTF8", F { return new FunctionPositionUTF8; });
|
||||||
|
factory.registerFunction("match", F { return new FunctionMatch; });
|
||||||
|
factory.registerFunction("like", F { return new FunctionLike; });
|
||||||
|
factory.registerFunction("notLike", F { return new FunctionNotLike; });
|
||||||
|
factory.registerFunction("extract", F { return new FunctionExtract; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
33
dbms/src/Functions/FunctionsURL.cpp
Normal file
33
dbms/src/Functions/FunctionsURL.cpp
Normal file
@ -0,0 +1,33 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsURL.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsURL(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("protocol", F { return new FunctionProtocol; });
|
||||||
|
factory.registerFunction("domain", F { return new FunctionDomain; });
|
||||||
|
factory.registerFunction("domainWithoutWWW", F { return new FunctionDomainWithoutWWW; });
|
||||||
|
factory.registerFunction("topLevelDomain", F { return new FunctionTopLevelDomain; });
|
||||||
|
factory.registerFunction("path", F { return new FunctionPath; });
|
||||||
|
factory.registerFunction("queryString", F { return new FunctionQueryString; });
|
||||||
|
factory.registerFunction("fragment", F { return new FunctionFragment; });
|
||||||
|
factory.registerFunction("queryStringAndFragment", F { return new FunctionQueryStringAndFragment; });
|
||||||
|
factory.registerFunction("extractURLParameter", F { return new FunctionExtractURLParameter; });
|
||||||
|
factory.registerFunction("extractURLParameters", F { return new FunctionExtractURLParameters; });
|
||||||
|
factory.registerFunction("extractURLParameterNames", F { return new FunctionExtractURLParameterNames; });
|
||||||
|
factory.registerFunction("URLHierarchy", F { return new FunctionURLHierarchy; });
|
||||||
|
factory.registerFunction("URLPathHierarchy", F { return new FunctionURLPathHierarchy; });
|
||||||
|
factory.registerFunction("cutWWW", F { return new FunctionCutWWW; });
|
||||||
|
factory.registerFunction("cutQueryString", F { return new FunctionCutQueryString; });
|
||||||
|
factory.registerFunction("cutFragment", F { return new FunctionCutFragment; });
|
||||||
|
factory.registerFunction("cutQueryStringAndFragment", F { return new FunctionCutQueryStringAndFragment; });
|
||||||
|
factory.registerFunction("cutURLParameter", F { return new FunctionCutURLParameter; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
22
dbms/src/Functions/FunctionsVisitParam.cpp
Normal file
22
dbms/src/Functions/FunctionsVisitParam.cpp
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Functions/FunctionsVisitParam.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionsVisitParam(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
#define F [](const Context & context) -> IFunction*
|
||||||
|
|
||||||
|
factory.registerFunction("visitParamHas", F { return new FunctionVisitParamHas; });
|
||||||
|
factory.registerFunction("visitParamExtractUInt", F { return new FunctionVisitParamExtractUInt; });
|
||||||
|
factory.registerFunction("visitParamExtractInt", F { return new FunctionVisitParamExtractInt; });
|
||||||
|
factory.registerFunction("visitParamExtractFloat", F { return new FunctionVisitParamExtractFloat; });
|
||||||
|
factory.registerFunction("visitParamExtractBool", F { return new FunctionVisitParamExtractBool; });
|
||||||
|
factory.registerFunction("visitParamExtractRaw", F { return new FunctionVisitParamExtractRaw; });
|
||||||
|
factory.registerFunction("visitParamExtractString", F { return new FunctionVisitParamExtractString; });
|
||||||
|
|
||||||
|
#undef F
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -105,7 +105,7 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
|||||||
* the first element of vector; otherwise we will just .emplace_back
|
* the first element of vector; otherwise we will just .emplace_back
|
||||||
*/
|
*/
|
||||||
std::vector<std::string> dir_names{};
|
std::vector<std::string> dir_names{};
|
||||||
auto has_local_node = false;
|
size_t num_local_nodes = 0;
|
||||||
|
|
||||||
auto first = true;
|
auto first = true;
|
||||||
for (auto jt = replica_keys.begin(); jt != replica_keys.end(); ++jt)
|
for (auto jt = replica_keys.begin(); jt != replica_keys.end(); ++jt)
|
||||||
@ -120,7 +120,7 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
|||||||
|
|
||||||
if (isLocal(replica_addresses.back()))
|
if (isLocal(replica_addresses.back()))
|
||||||
{
|
{
|
||||||
has_local_node = true;
|
++num_local_nodes;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -143,7 +143,7 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
|||||||
}
|
}
|
||||||
|
|
||||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
|
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
|
||||||
shard_info_vec.push_back({std::move(dir_names), weight, has_local_node});
|
shard_info_vec.push_back({std::move(dir_names), weight, num_local_nodes});
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
#include <DB/Interpreters/Context.h>
|
#include <DB/Interpreters/Context.h>
|
||||||
#include <DB/Client/ConnectionPoolWithFailover.h>
|
#include <DB/Client/ConnectionPoolWithFailover.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -448,17 +449,6 @@ void Context::setDefaultFormat(const String & name)
|
|||||||
default_format = name;
|
default_format = name;
|
||||||
}
|
}
|
||||||
|
|
||||||
String Context::getDefaultReplicaName() const
|
|
||||||
{
|
|
||||||
return shared->default_replica_name;
|
|
||||||
}
|
|
||||||
|
|
||||||
void Context::setDefaultReplicaName(const String & name)
|
|
||||||
{
|
|
||||||
/// Полагаемся, что это присваивание происходит один раз при старте сервера. Если это не так, нужно использовать мьютекс.
|
|
||||||
shared->default_replica_name = name;
|
|
||||||
}
|
|
||||||
|
|
||||||
const Macros& Context::getMacros() const
|
const Macros& Context::getMacros() const
|
||||||
{
|
{
|
||||||
return shared->macros;
|
return shared->macros;
|
||||||
|
@ -85,86 +85,124 @@ void ExpressionAction::prepare(Block & sample_block)
|
|||||||
{
|
{
|
||||||
// std::cerr << "preparing: " << toString() << std::endl;
|
// std::cerr << "preparing: " << toString() << std::endl;
|
||||||
|
|
||||||
if (type == APPLY_FUNCTION)
|
switch (type)
|
||||||
{
|
{
|
||||||
if (sample_block.has(result_name))
|
case APPLY_FUNCTION:
|
||||||
throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN);
|
|
||||||
|
|
||||||
bool all_const = true;
|
|
||||||
|
|
||||||
ColumnNumbers arguments(argument_names.size());
|
|
||||||
for (size_t i = 0; i < argument_names.size(); ++i)
|
|
||||||
{
|
{
|
||||||
arguments[i] = sample_block.getPositionByName(argument_names[i]);
|
if (sample_block.has(result_name))
|
||||||
ColumnPtr col = sample_block.getByPosition(arguments[i]).column;
|
throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN);
|
||||||
if (!col || !col->isConst())
|
|
||||||
all_const = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnNumbers prerequisites(prerequisite_names.size());
|
bool all_const = true;
|
||||||
for (size_t i = 0; i < prerequisite_names.size(); ++i)
|
|
||||||
{
|
|
||||||
prerequisites[i] = sample_block.getPositionByName(prerequisite_names[i]);
|
|
||||||
ColumnPtr col = sample_block.getByPosition(prerequisites[i]).column;
|
|
||||||
if (!col || !col->isConst())
|
|
||||||
all_const = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr new_column;
|
ColumnNumbers arguments(argument_names.size());
|
||||||
|
for (size_t i = 0; i < argument_names.size(); ++i)
|
||||||
/// Если все аргументы и требуемые столбцы - константы, выполним функцию.
|
|
||||||
if (all_const)
|
|
||||||
{
|
|
||||||
ColumnWithNameAndType new_column;
|
|
||||||
new_column.name = result_name;
|
|
||||||
new_column.type = result_type;
|
|
||||||
sample_block.insert(new_column);
|
|
||||||
|
|
||||||
size_t result_position = sample_block.getPositionByName(result_name);
|
|
||||||
function->execute(sample_block, arguments, prerequisites, result_position);
|
|
||||||
|
|
||||||
/// Если получилась не константа, на всякий случай будем считать результат неизвестным.
|
|
||||||
ColumnWithNameAndType & col = sample_block.getByPosition(result_position);
|
|
||||||
if (!col.column->isConst())
|
|
||||||
{
|
{
|
||||||
col.column = nullptr;
|
arguments[i] = sample_block.getPositionByName(argument_names[i]);
|
||||||
|
ColumnPtr col = sample_block.getByPosition(arguments[i]).column;
|
||||||
|
if (!col || !col->isConst())
|
||||||
|
all_const = false;
|
||||||
}
|
}
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
sample_block.insert(ColumnWithNameAndType(nullptr, result_type, result_name));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else if (type == ARRAY_JOIN)
|
|
||||||
{
|
|
||||||
for (NameSet::iterator it = array_joined_columns.begin(); it != array_joined_columns.end(); ++it)
|
|
||||||
{
|
|
||||||
ColumnWithNameAndType & current = sample_block.getByName(*it);
|
|
||||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*current.type);
|
|
||||||
if (!array_type)
|
|
||||||
throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH);
|
|
||||||
current.type = array_type->getNestedType();
|
|
||||||
current.column = nullptr;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else if (type == JOIN)
|
|
||||||
{
|
|
||||||
for (const auto & col : columns_added_by_join)
|
|
||||||
sample_block.insert(ColumnWithNameAndType(col.type->createColumn(), col.type, col.name));
|
|
||||||
}
|
|
||||||
else if (type == ADD_COLUMN)
|
|
||||||
{
|
|
||||||
if (sample_block.has(result_name))
|
|
||||||
throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN);
|
|
||||||
|
|
||||||
sample_block.insert(ColumnWithNameAndType(added_column, result_type, result_name));
|
ColumnNumbers prerequisites(prerequisite_names.size());
|
||||||
}
|
for (size_t i = 0; i < prerequisite_names.size(); ++i)
|
||||||
else
|
{
|
||||||
{
|
prerequisites[i] = sample_block.getPositionByName(prerequisite_names[i]);
|
||||||
if (type == COPY_COLUMN)
|
ColumnPtr col = sample_block.getByPosition(prerequisites[i]).column;
|
||||||
|
if (!col || !col->isConst())
|
||||||
|
all_const = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr new_column;
|
||||||
|
|
||||||
|
/// Если все аргументы и требуемые столбцы - константы, выполним функцию.
|
||||||
|
if (all_const)
|
||||||
|
{
|
||||||
|
ColumnWithNameAndType new_column;
|
||||||
|
new_column.name = result_name;
|
||||||
|
new_column.type = result_type;
|
||||||
|
sample_block.insert(new_column);
|
||||||
|
|
||||||
|
size_t result_position = sample_block.getPositionByName(result_name);
|
||||||
|
function->execute(sample_block, arguments, prerequisites, result_position);
|
||||||
|
|
||||||
|
/// Если получилась не константа, на всякий случай будем считать результат неизвестным.
|
||||||
|
ColumnWithNameAndType & col = sample_block.getByPosition(result_position);
|
||||||
|
if (!col.column->isConst())
|
||||||
|
col.column = nullptr;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
sample_block.insert(ColumnWithNameAndType(nullptr, result_type, result_name));
|
||||||
|
}
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case ARRAY_JOIN:
|
||||||
|
{
|
||||||
|
for (NameSet::iterator it = array_joined_columns.begin(); it != array_joined_columns.end(); ++it)
|
||||||
|
{
|
||||||
|
ColumnWithNameAndType & current = sample_block.getByName(*it);
|
||||||
|
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*current.type);
|
||||||
|
if (!array_type)
|
||||||
|
throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH);
|
||||||
|
current.type = array_type->getNestedType();
|
||||||
|
current.column = nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case JOIN:
|
||||||
|
{
|
||||||
|
for (const auto & col : columns_added_by_join)
|
||||||
|
sample_block.insert(ColumnWithNameAndType(col.type->createColumn(), col.type, col.name));
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case PROJECT:
|
||||||
|
{
|
||||||
|
Block new_block;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < projection.size(); ++i)
|
||||||
|
{
|
||||||
|
const std::string & name = projection[i].first;
|
||||||
|
const std::string & alias = projection[i].second;
|
||||||
|
ColumnWithNameAndType column = sample_block.getByName(name);
|
||||||
|
if (alias != "")
|
||||||
|
column.name = alias;
|
||||||
|
new_block.insert(column);
|
||||||
|
}
|
||||||
|
|
||||||
|
sample_block.swap(new_block);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case REMOVE_COLUMN:
|
||||||
|
{
|
||||||
|
sample_block.erase(source_name);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case ADD_COLUMN:
|
||||||
|
{
|
||||||
|
if (sample_block.has(result_name))
|
||||||
|
throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN);
|
||||||
|
|
||||||
|
sample_block.insert(ColumnWithNameAndType(added_column, result_type, result_name));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case COPY_COLUMN:
|
||||||
|
{
|
||||||
result_type = sample_block.getByName(source_name).type;
|
result_type = sample_block.getByName(source_name).type;
|
||||||
|
sample_block.insert(ColumnWithNameAndType(sample_block.getByName(source_name).column, result_type, result_name));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
execute(sample_block);
|
default:
|
||||||
|
throw Exception("Unknown action type", ErrorCodes::UNKNOWN_ACTION);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -366,24 +404,27 @@ void ExpressionActions::checkLimits(Block & block) const
|
|||||||
const Limits & limits = settings.limits;
|
const Limits & limits = settings.limits;
|
||||||
if (limits.max_temporary_columns && block.columns() > limits.max_temporary_columns)
|
if (limits.max_temporary_columns && block.columns() > limits.max_temporary_columns)
|
||||||
throw Exception("Too many temporary columns: " + block.dumpNames()
|
throw Exception("Too many temporary columns: " + block.dumpNames()
|
||||||
+ ". Maximum: " + toString(limits.max_temporary_columns),
|
+ ". Maximum: " + toString(limits.max_temporary_columns),
|
||||||
ErrorCodes::TOO_MUCH_TEMPORARY_COLUMNS);
|
ErrorCodes::TOO_MUCH_TEMPORARY_COLUMNS);
|
||||||
|
|
||||||
size_t non_const_columns = 0;
|
if (limits.max_temporary_non_const_columns)
|
||||||
for (size_t i = 0, size = block.columns(); i < size; ++i)
|
|
||||||
if (block.getByPosition(i).column && !block.getByPosition(i).column->isConst())
|
|
||||||
++non_const_columns;
|
|
||||||
|
|
||||||
if (limits.max_temporary_non_const_columns && non_const_columns > limits.max_temporary_non_const_columns)
|
|
||||||
{
|
{
|
||||||
std::stringstream list_of_non_const_columns;
|
size_t non_const_columns = 0;
|
||||||
for (size_t i = 0, size = block.columns(); i < size; ++i)
|
for (size_t i = 0, size = block.columns(); i < size; ++i)
|
||||||
if (!block.getByPosition(i).column->isConst())
|
if (block.getByPosition(i).column && !block.getByPosition(i).column->isConst())
|
||||||
list_of_non_const_columns << (i == 0 ? "" : ", ") << block.getByPosition(i).name;
|
++non_const_columns;
|
||||||
|
|
||||||
throw Exception("Too many temporary non-const columns: " + list_of_non_const_columns.str()
|
if (non_const_columns > limits.max_temporary_non_const_columns)
|
||||||
+ ". Maximum: " + toString(limits.max_temporary_non_const_columns),
|
{
|
||||||
ErrorCodes::TOO_MUCH_TEMPORARY_NON_CONST_COLUMNS);
|
std::stringstream list_of_non_const_columns;
|
||||||
|
for (size_t i = 0, size = block.columns(); i < size; ++i)
|
||||||
|
if (!block.getByPosition(i).column->isConst())
|
||||||
|
list_of_non_const_columns << (i == 0 ? "" : ", ") << block.getByPosition(i).name;
|
||||||
|
|
||||||
|
throw Exception("Too many temporary non-const columns: " + list_of_non_const_columns.str()
|
||||||
|
+ ". Maximum: " + toString(limits.max_temporary_non_const_columns),
|
||||||
|
ErrorCodes::TOO_MUCH_TEMPORARY_NON_CONST_COLUMNS);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -600,7 +641,6 @@ void ExpressionActions::finalize(const Names & output_columns)
|
|||||||
}
|
}
|
||||||
|
|
||||||
unmodified_columns.erase(out);
|
unmodified_columns.erase(out);
|
||||||
|
|
||||||
needed_columns.erase(out);
|
needed_columns.erase(out);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -639,6 +679,7 @@ void ExpressionActions::finalize(const Names & output_columns)
|
|||||||
checkLimits(sample_block);
|
checkLimits(sample_block);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
std::string ExpressionActions::getID() const
|
std::string ExpressionActions::getID() const
|
||||||
{
|
{
|
||||||
std::stringstream ss;
|
std::stringstream ss;
|
||||||
|
@ -238,50 +238,52 @@ BlockInputStreamPtr InterpreterSelectQuery::execute()
|
|||||||
* выбрасывать ненужные столбцы с учетом всего запроса. В ненужных частях запроса не будем выполнять подзапросы.
|
* выбрасывать ненужные столбцы с учетом всего запроса. В ненужных частях запроса не будем выполнять подзапросы.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
ExpressionActionsChain chain;
|
|
||||||
|
|
||||||
need_aggregate = query_analyzer->hasAggregation();
|
|
||||||
|
|
||||||
query_analyzer->appendArrayJoin(chain, !first_stage);
|
|
||||||
query_analyzer->appendJoin(chain, !first_stage);
|
|
||||||
|
|
||||||
if (query_analyzer->appendWhere(chain, !first_stage))
|
|
||||||
{
|
{
|
||||||
has_where = true;
|
ExpressionActionsChain chain;
|
||||||
before_where = chain.getLastActions();
|
|
||||||
|
need_aggregate = query_analyzer->hasAggregation();
|
||||||
|
|
||||||
|
query_analyzer->appendArrayJoin(chain, !first_stage);
|
||||||
|
query_analyzer->appendJoin(chain, !first_stage);
|
||||||
|
|
||||||
|
if (query_analyzer->appendWhere(chain, !first_stage))
|
||||||
|
{
|
||||||
|
has_where = true;
|
||||||
|
before_where = chain.getLastActions();
|
||||||
|
chain.addStep();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (need_aggregate)
|
||||||
|
{
|
||||||
|
query_analyzer->appendGroupBy(chain, !first_stage);
|
||||||
|
query_analyzer->appendAggregateFunctionsArguments(chain, !first_stage);
|
||||||
|
before_aggregation = chain.getLastActions();
|
||||||
|
|
||||||
|
chain.finalize();
|
||||||
|
chain.clear();
|
||||||
|
|
||||||
|
if (query_analyzer->appendHaving(chain, !second_stage))
|
||||||
|
{
|
||||||
|
has_having = true;
|
||||||
|
before_having = chain.getLastActions();
|
||||||
|
chain.addStep();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Если есть агрегация, выполняем выражения в SELECT и ORDER BY на инициировавшем сервере, иначе - на серверах-источниках.
|
||||||
|
query_analyzer->appendSelect(chain, need_aggregate ? !second_stage : !first_stage);
|
||||||
|
selected_columns = chain.getLastStep().required_output;
|
||||||
|
has_order_by = query_analyzer->appendOrderBy(chain, need_aggregate ? !second_stage : !first_stage);
|
||||||
|
before_order_and_select = chain.getLastActions();
|
||||||
chain.addStep();
|
chain.addStep();
|
||||||
}
|
|
||||||
|
|
||||||
if (need_aggregate)
|
query_analyzer->appendProjectResult(chain, !second_stage);
|
||||||
{
|
final_projection = chain.getLastActions();
|
||||||
query_analyzer->appendGroupBy(chain, !first_stage);
|
|
||||||
query_analyzer->appendAggregateFunctionsArguments(chain, !first_stage);
|
|
||||||
before_aggregation = chain.getLastActions();
|
|
||||||
|
|
||||||
chain.finalize();
|
chain.finalize();
|
||||||
chain.clear();
|
chain.clear();
|
||||||
|
|
||||||
if (query_analyzer->appendHaving(chain, !second_stage))
|
|
||||||
{
|
|
||||||
has_having = true;
|
|
||||||
before_having = chain.getLastActions();
|
|
||||||
chain.addStep();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Если есть агрегация, выполняем выражения в SELECT и ORDER BY на инициировавшем сервере, иначе - на серверах-источниках.
|
|
||||||
query_analyzer->appendSelect(chain, need_aggregate ? !second_stage : !first_stage);
|
|
||||||
selected_columns = chain.getLastStep().required_output;
|
|
||||||
has_order_by = query_analyzer->appendOrderBy(chain, need_aggregate ? !second_stage : !first_stage);
|
|
||||||
before_order_and_select = chain.getLastActions();
|
|
||||||
chain.addStep();
|
|
||||||
|
|
||||||
query_analyzer->appendProjectResult(chain, !second_stage);
|
|
||||||
final_projection = chain.getLastActions();
|
|
||||||
|
|
||||||
chain.finalize();
|
|
||||||
chain.clear();
|
|
||||||
|
|
||||||
/// Перед выполнением HAVING уберем из блока лишние столбцы (в основном, ключи агрегации).
|
/// Перед выполнением HAVING уберем из блока лишние столбцы (в основном, ключи агрегации).
|
||||||
if (has_having)
|
if (has_having)
|
||||||
before_having->prependProjectInput();
|
before_having->prependProjectInput();
|
||||||
|
@ -1,7 +1,5 @@
|
|||||||
#include <iomanip>
|
#include <iomanip>
|
||||||
|
|
||||||
#include <statdaemons/Stopwatch.h>
|
|
||||||
|
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
#include <DB/Columns/ColumnFixedString.h>
|
#include <DB/Columns/ColumnFixedString.h>
|
||||||
|
|
||||||
@ -14,8 +12,6 @@ namespace DB
|
|||||||
|
|
||||||
void SplittingAggregator::execute(BlockInputStreamPtr stream, ManyAggregatedDataVariants & results)
|
void SplittingAggregator::execute(BlockInputStreamPtr stream, ManyAggregatedDataVariants & results)
|
||||||
{
|
{
|
||||||
//Stopwatch watch;
|
|
||||||
|
|
||||||
/// Читаем все данные
|
/// Читаем все данные
|
||||||
while (Block block = stream->read())
|
while (Block block = stream->read())
|
||||||
{
|
{
|
||||||
@ -90,10 +86,6 @@ void SplittingAggregator::execute(BlockInputStreamPtr stream, ManyAggregatedData
|
|||||||
|
|
||||||
/// Параллельно вычисляем хэши и ключи.
|
/// Параллельно вычисляем хэши и ключи.
|
||||||
|
|
||||||
// LOG_TRACE(log, "Calculating keys and hashes.");
|
|
||||||
|
|
||||||
// watch.start();
|
|
||||||
|
|
||||||
for (size_t thread_no = 0; thread_no < threads; ++thread_no)
|
for (size_t thread_no = 0; thread_no < threads; ++thread_no)
|
||||||
pool.schedule(boost::bind(&SplittingAggregator::calculateHashesThread, this,
|
pool.schedule(boost::bind(&SplittingAggregator::calculateHashesThread, this,
|
||||||
boost::ref(block),
|
boost::ref(block),
|
||||||
@ -106,13 +98,8 @@ void SplittingAggregator::execute(BlockInputStreamPtr stream, ManyAggregatedData
|
|||||||
|
|
||||||
rethrowFirstException(exceptions);
|
rethrowFirstException(exceptions);
|
||||||
|
|
||||||
// LOG_TRACE(log, "Calculated keys and hashes in " << std::fixed << std::setprecision(2) << watch.elapsedSeconds() << " sec.");
|
|
||||||
// watch.restart();
|
|
||||||
|
|
||||||
/// Параллельно агрегируем в независимые хэш-таблицы
|
/// Параллельно агрегируем в независимые хэш-таблицы
|
||||||
|
|
||||||
// LOG_TRACE(log, "Parallel aggregating.");
|
|
||||||
|
|
||||||
for (size_t thread_no = 0; thread_no < threads; ++thread_no)
|
for (size_t thread_no = 0; thread_no < threads; ++thread_no)
|
||||||
pool.schedule(boost::bind(&SplittingAggregator::aggregateThread, this,
|
pool.schedule(boost::bind(&SplittingAggregator::aggregateThread, this,
|
||||||
boost::ref(block),
|
boost::ref(block),
|
||||||
@ -125,8 +112,6 @@ void SplittingAggregator::execute(BlockInputStreamPtr stream, ManyAggregatedData
|
|||||||
|
|
||||||
rethrowFirstException(exceptions);
|
rethrowFirstException(exceptions);
|
||||||
|
|
||||||
// LOG_TRACE(log, "Parallel aggregated in " << std::fixed << std::setprecision(2) << watch.elapsedSeconds() << " sec.");
|
|
||||||
|
|
||||||
/// Проверка ограничений
|
/// Проверка ограничений
|
||||||
|
|
||||||
if (max_rows_to_group_by && size_of_all_results > max_rows_to_group_by && group_by_overflow_mode == OverflowMode::BREAK)
|
if (max_rows_to_group_by && size_of_all_results > max_rows_to_group_by && group_by_overflow_mode == OverflowMode::BREAK)
|
||||||
|
@ -370,9 +370,6 @@ int Server::main(const std::vector<std::string> & args)
|
|||||||
global_context->setInterserverIOHost(this_host, port);
|
global_context->setInterserverIOHost(this_host, port);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (config().has("replica_name"))
|
|
||||||
global_context->setDefaultReplicaName(config().getString("replica_name"));
|
|
||||||
|
|
||||||
if (config().has("macros"))
|
if (config().has("macros"))
|
||||||
global_context->setMacros(Macros(config(), "macros"));
|
global_context->setMacros(Macros(config(), "macros"));
|
||||||
|
|
||||||
@ -405,6 +402,7 @@ int Server::main(const std::vector<std::string> & args)
|
|||||||
|
|
||||||
global_context->addTable("system", "one", StorageSystemOne::create("one"));
|
global_context->addTable("system", "one", StorageSystemOne::create("one"));
|
||||||
global_context->addTable("system", "numbers", StorageSystemNumbers::create("numbers"));
|
global_context->addTable("system", "numbers", StorageSystemNumbers::create("numbers"));
|
||||||
|
global_context->addTable("system", "numbers_mt", StorageSystemNumbers::create("numbers_mt", true));
|
||||||
global_context->addTable("system", "tables", StorageSystemTables::create("tables", *global_context));
|
global_context->addTable("system", "tables", StorageSystemTables::create("tables", *global_context));
|
||||||
global_context->addTable("system", "parts", StorageSystemParts::create("parts", *global_context));
|
global_context->addTable("system", "parts", StorageSystemParts::create("parts", *global_context));
|
||||||
global_context->addTable("system", "databases", StorageSystemDatabases::create("databases", *global_context));
|
global_context->addTable("system", "databases", StorageSystemDatabases::create("databases", *global_context));
|
||||||
|
@ -42,7 +42,7 @@ void TCPHandler::runImpl()
|
|||||||
socket().setReceiveTimeout(global_settings.receive_timeout);
|
socket().setReceiveTimeout(global_settings.receive_timeout);
|
||||||
socket().setSendTimeout(global_settings.send_timeout);
|
socket().setSendTimeout(global_settings.send_timeout);
|
||||||
socket().setNoDelay(true);
|
socket().setNoDelay(true);
|
||||||
|
|
||||||
in = new ReadBufferFromPocoSocket(socket());
|
in = new ReadBufferFromPocoSocket(socket());
|
||||||
out = new WriteBufferFromPocoSocket(socket());
|
out = new WriteBufferFromPocoSocket(socket());
|
||||||
|
|
||||||
@ -82,7 +82,7 @@ void TCPHandler::runImpl()
|
|||||||
|
|
||||||
connection_context.setCurrentDatabase(default_database);
|
connection_context.setCurrentDatabase(default_database);
|
||||||
}
|
}
|
||||||
|
|
||||||
sendHello();
|
sendHello();
|
||||||
|
|
||||||
connection_context.setProgressCallback([this] (const size_t rows, const size_t bytes) {
|
connection_context.setProgressCallback([this] (const size_t rows, const size_t bytes) {
|
||||||
@ -98,7 +98,7 @@ void TCPHandler::runImpl()
|
|||||||
/// Если требуется завершить работу, или клиент отсоединился.
|
/// Если требуется завершить работу, или клиент отсоединился.
|
||||||
if (Daemon::instance().isCancelled() || in->eof())
|
if (Daemon::instance().isCancelled() || in->eof())
|
||||||
break;
|
break;
|
||||||
|
|
||||||
Stopwatch watch;
|
Stopwatch watch;
|
||||||
state.reset();
|
state.reset();
|
||||||
|
|
||||||
@ -106,7 +106,7 @@ void TCPHandler::runImpl()
|
|||||||
* Клиент сможет его принять, если оно не произошло во время отправки другого пакета и клиент ещё не разорвал соединение.
|
* Клиент сможет его принять, если оно не произошло во время отправки другого пакета и клиент ещё не разорвал соединение.
|
||||||
*/
|
*/
|
||||||
SharedPtr<Exception> exception;
|
SharedPtr<Exception> exception;
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
/// Восстанавливаем контекст запроса.
|
/// Восстанавливаем контекст запроса.
|
||||||
@ -271,7 +271,7 @@ void TCPHandler::processOrdinaryQuery()
|
|||||||
while (true)
|
while (true)
|
||||||
{
|
{
|
||||||
Block block;
|
Block block;
|
||||||
|
|
||||||
while (true)
|
while (true)
|
||||||
{
|
{
|
||||||
if (isQueryCancelled())
|
if (isQueryCancelled())
|
||||||
@ -288,7 +288,7 @@ void TCPHandler::processOrdinaryQuery()
|
|||||||
after_send_progress.restart();
|
after_send_progress.restart();
|
||||||
sendProgress();
|
sendProgress();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (async_in.poll(query_context.getSettingsRef().interactive_delay / 1000))
|
if (async_in.poll(query_context.getSettingsRef().interactive_delay / 1000))
|
||||||
{
|
{
|
||||||
/// Есть следующий блок результата.
|
/// Есть следующий блок результата.
|
||||||
@ -308,8 +308,8 @@ void TCPHandler::processOrdinaryQuery()
|
|||||||
sendProfileInfo();
|
sendProfileInfo();
|
||||||
sendProgress();
|
sendProgress();
|
||||||
}
|
}
|
||||||
|
|
||||||
sendData(block);
|
sendData(block);
|
||||||
if (!block)
|
if (!block)
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -20,35 +20,30 @@ namespace DB
|
|||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
template <typename ASTType> void rewriteImpl(ASTType &, const std::string &, const std::string &) = delete;
|
|
||||||
|
|
||||||
/// select query has database and table names as AST pointers
|
/// select query has database and table names as AST pointers
|
||||||
template <> inline void rewriteImpl<ASTSelectQuery>(ASTSelectQuery & query,
|
/// Создает копию запроса, меняет имена базы данных и таблицы.
|
||||||
const std::string & database, const std::string & table)
|
inline ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, const std::string & table)
|
||||||
{
|
{
|
||||||
query.database = new ASTIdentifier{{}, database, ASTIdentifier::Database};
|
auto modified_query_ast = query->clone();
|
||||||
query.table = new ASTIdentifier{{}, table, ASTIdentifier::Table};
|
|
||||||
|
auto & actual_query = typeid_cast<ASTSelectQuery &>(*modified_query_ast);
|
||||||
|
actual_query.database = new ASTIdentifier{{}, database, ASTIdentifier::Database};
|
||||||
|
actual_query.table = new ASTIdentifier{{}, table, ASTIdentifier::Table};
|
||||||
|
|
||||||
|
return modified_query_ast;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// insert query has database and table names as bare strings
|
/// insert query has database and table names as bare strings
|
||||||
template <> inline void rewriteImpl<ASTInsertQuery>(ASTInsertQuery & query,
|
|
||||||
const std::string & database, const std::string & table)
|
|
||||||
{
|
|
||||||
query.database = database;
|
|
||||||
query.table = table;
|
|
||||||
/// make sure query is not INSERT SELECT
|
|
||||||
query.select = nullptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Создает копию запроса, меняет имена базы данных и таблицы.
|
/// Создает копию запроса, меняет имена базы данных и таблицы.
|
||||||
template <typename ASTType>
|
inline ASTPtr rewriteInsertQuery(const ASTPtr & query, const std::string & database, const std::string & table)
|
||||||
inline ASTPtr rewriteQuery(const ASTPtr & query, const std::string & database, const std::string & table)
|
|
||||||
{
|
{
|
||||||
/// Создаем копию запроса.
|
|
||||||
auto modified_query_ast = query->clone();
|
auto modified_query_ast = query->clone();
|
||||||
|
|
||||||
/// Меняем имена таблицы и базы данных
|
auto & actual_query = typeid_cast<ASTInsertQuery &>(*modified_query_ast);
|
||||||
rewriteImpl(typeid_cast<ASTType &>(*modified_query_ast), database, table);
|
actual_query.database = database;
|
||||||
|
actual_query.table = table;
|
||||||
|
/// make sure query is not INSERT SELECT
|
||||||
|
actual_query.select = nullptr;
|
||||||
|
|
||||||
return modified_query_ast;
|
return modified_query_ast;
|
||||||
}
|
}
|
||||||
@ -131,9 +126,9 @@ BlockInputStreams StorageDistributed::read(
|
|||||||
: QueryProcessingStage::WithMergeableState;
|
: QueryProcessingStage::WithMergeableState;
|
||||||
|
|
||||||
BlockInputStreams res;
|
BlockInputStreams res;
|
||||||
const auto & modified_query_ast = rewriteQuery<ASTSelectQuery>(
|
const auto & modified_query_ast = rewriteSelectQuery(
|
||||||
query, remote_database, remote_table);
|
query, remote_database, remote_table);
|
||||||
const auto & modified_query = queryToString<ASTSelectQuery>(modified_query_ast);
|
const auto & modified_query = queryToString(modified_query_ast);
|
||||||
|
|
||||||
/// Цикл по шардам.
|
/// Цикл по шардам.
|
||||||
for (auto & conn_pool : cluster.pools)
|
for (auto & conn_pool : cluster.pools)
|
||||||
@ -172,7 +167,7 @@ BlockOutputStreamPtr StorageDistributed::write(ASTPtr query)
|
|||||||
|
|
||||||
return new DistributedBlockOutputStream{
|
return new DistributedBlockOutputStream{
|
||||||
*this,
|
*this,
|
||||||
rewriteQuery<ASTInsertQuery>(query, remote_database, remote_table)
|
rewriteInsertQuery(query, remote_database, remote_table)
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -268,11 +268,7 @@ StoragePtr StorageFactory::get(
|
|||||||
throw Exception("Replica name must be a string literal", ErrorCodes::BAD_ARGUMENTS);
|
throw Exception("Replica name must be a string literal", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
if (replica_name.empty())
|
if (replica_name.empty())
|
||||||
{
|
throw Exception("No replica name in config", ErrorCodes::NO_REPLICA_NAME_GIVEN);
|
||||||
replica_name = context.getDefaultReplicaName();
|
|
||||||
if (replica_name.empty())
|
|
||||||
throw Exception("No replica name in config", ErrorCodes::NO_REPLICA_NAME_GIVEN);
|
|
||||||
}
|
|
||||||
|
|
||||||
args.erase(args.begin(), args.begin() + 2);
|
args.erase(args.begin(), args.begin() + 2);
|
||||||
}
|
}
|
||||||
@ -315,22 +311,6 @@ StoragePtr StorageFactory::get(
|
|||||||
columns, context, primary_expr_list, date_column_name,
|
columns, context, primary_expr_list, date_column_name,
|
||||||
sampling_expression, index_granularity, mode, sign_column_name);
|
sampling_expression, index_granularity, mode, sign_column_name);
|
||||||
}
|
}
|
||||||
else if (name == "SystemNumbers")
|
|
||||||
{
|
|
||||||
if (columns->size() != 1 || columns->begin()->name != "number" || columns->begin()->type->getName() != "UInt64")
|
|
||||||
throw Exception("Storage SystemNumbers only allows one column with name 'number' and type 'UInt64'",
|
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
|
|
||||||
return StorageSystemNumbers::create(table_name);
|
|
||||||
}
|
|
||||||
else if (name == "SystemOne")
|
|
||||||
{
|
|
||||||
if (columns->size() != 1 || columns->begin()->name != "dummy" || columns->begin()->type->getName() != "UInt8")
|
|
||||||
throw Exception("Storage SystemOne only allows one column with name 'dummy' and type 'UInt8'",
|
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
|
|
||||||
return StorageSystemOne::create(table_name);
|
|
||||||
}
|
|
||||||
else
|
else
|
||||||
throw Exception("Unknown storage " + name, ErrorCodes::UNKNOWN_STORAGE);
|
throw Exception("Unknown storage " + name, ErrorCodes::UNKNOWN_STORAGE);
|
||||||
}
|
}
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <DB/Core/ErrorCodes.h>
|
#include <DB/Core/ErrorCodes.h>
|
||||||
#include <DB/Columns/ColumnsNumber.h>
|
#include <DB/Columns/ColumnsNumber.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/StorageSystemNumbers.h>
|
||||||
|
|
||||||
|
|
||||||
@ -12,46 +13,55 @@ namespace DB
|
|||||||
|
|
||||||
using Poco::SharedPtr;
|
using Poco::SharedPtr;
|
||||||
|
|
||||||
|
class NumbersBlockInputStream : public IProfilingBlockInputStream
|
||||||
NumbersBlockInputStream::NumbersBlockInputStream(size_t block_size_) : block_size(block_size_), next(0)
|
|
||||||
{
|
{
|
||||||
}
|
public:
|
||||||
|
NumbersBlockInputStream(size_t block_size_, size_t offset_, size_t step_)
|
||||||
|
: block_size(block_size_), next(offset_), step(step_) {}
|
||||||
|
|
||||||
|
String getName() const { return "NumbersBlockInputStream"; }
|
||||||
|
String getID() const { return "Numbers"; }
|
||||||
|
|
||||||
|
protected:
|
||||||
|
Block readImpl()
|
||||||
|
{
|
||||||
|
Block res;
|
||||||
|
|
||||||
|
ColumnWithNameAndType column_with_name_and_type;
|
||||||
|
|
||||||
|
column_with_name_and_type.name = "number";
|
||||||
|
column_with_name_and_type.type = new DataTypeUInt64();
|
||||||
|
ColumnUInt64 * column = new ColumnUInt64(block_size);
|
||||||
|
ColumnUInt64::Container_t & vec = column->getData();
|
||||||
|
column_with_name_and_type.column = column;
|
||||||
|
|
||||||
|
size_t curr = next; /// Локальная переменная почему-то работает быстрее (>20%), чем член класса.
|
||||||
|
UInt64 * pos = &vec[0]; /// Это тоже ускоряет код.
|
||||||
|
UInt64 * end = &vec[block_size];
|
||||||
|
while (pos < end)
|
||||||
|
*pos++ = curr++;
|
||||||
|
|
||||||
|
res.insert(column_with_name_and_type);
|
||||||
|
|
||||||
|
next += step;
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
private:
|
||||||
|
size_t block_size;
|
||||||
|
UInt64 next;
|
||||||
|
UInt64 step;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
Block NumbersBlockInputStream::readImpl()
|
StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_)
|
||||||
{
|
: name(name_), multithreaded(multithreaded_)
|
||||||
Block res;
|
|
||||||
|
|
||||||
ColumnWithNameAndType column_with_name_and_type;
|
|
||||||
|
|
||||||
column_with_name_and_type.name = "number";
|
|
||||||
column_with_name_and_type.type = new DataTypeUInt64();
|
|
||||||
ColumnUInt64 * column = new ColumnUInt64(block_size);
|
|
||||||
ColumnUInt64::Container_t & vec = column->getData();
|
|
||||||
column_with_name_and_type.column = column;
|
|
||||||
|
|
||||||
size_t curr = next; /// Локальная переменная почему-то работает быстрее (>20%), чем член класса.
|
|
||||||
UInt64 * pos = &vec[0]; /// Это тоже ускоряет код.
|
|
||||||
UInt64 * end = &vec[block_size];
|
|
||||||
while (pos < end)
|
|
||||||
*pos++ = curr++;
|
|
||||||
next = curr;
|
|
||||||
|
|
||||||
res.insert(column_with_name_and_type);
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
StorageSystemNumbers::StorageSystemNumbers(const std::string & name_)
|
|
||||||
: name(name_)
|
|
||||||
{
|
{
|
||||||
columns.push_back(NameAndTypePair("number", new DataTypeUInt64));
|
columns.push_back(NameAndTypePair("number", new DataTypeUInt64));
|
||||||
}
|
}
|
||||||
|
|
||||||
StoragePtr StorageSystemNumbers::create(const std::string & name_)
|
StoragePtr StorageSystemNumbers::create(const std::string & name_, bool multithreaded_)
|
||||||
{
|
{
|
||||||
return (new StorageSystemNumbers(name_))->thisPtr();
|
return (new StorageSystemNumbers(name_, multithreaded_))->thisPtr();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -61,7 +71,15 @@ BlockInputStreams StorageSystemNumbers::read(
|
|||||||
{
|
{
|
||||||
check(column_names);
|
check(column_names);
|
||||||
processed_stage = QueryProcessingStage::FetchColumns;
|
processed_stage = QueryProcessingStage::FetchColumns;
|
||||||
return BlockInputStreams(1, new NumbersBlockInputStream(max_block_size));
|
|
||||||
|
if (!multithreaded)
|
||||||
|
threads = 1;
|
||||||
|
|
||||||
|
BlockInputStreams res(threads);
|
||||||
|
for (size_t i = 0; i < threads; ++i)
|
||||||
|
res[i] = new NumbersBlockInputStream(max_block_size, i * max_block_size, threads * max_block_size);
|
||||||
|
|
||||||
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user