mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge
This commit is contained in:
commit
b7092a7ab1
120
dbms/include/DB/AggregateFunctions/AggregateFunctionArray.h
Normal file
120
dbms/include/DB/AggregateFunctions/AggregateFunctionArray.h
Normal file
@ -0,0 +1,120 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Не агрегатная функция, а адаптер агрегатных функций,
|
||||
* который любую агрегатную функцию agg(x) делает агрегатной функцией вида aggArray(x).
|
||||
* Адаптированная агрегатная функция вычисляет вложенную агрегатную функцию для каждого элемента массива.
|
||||
*/
|
||||
class AggregateFunctionArray : public IAggregateFunction
|
||||
{
|
||||
private:
|
||||
AggregateFunctionPtr nested_func_owner;
|
||||
IAggregateFunction * nested_func;
|
||||
int num_agruments;
|
||||
|
||||
public:
|
||||
AggregateFunctionArray(AggregateFunctionPtr nested_) : nested_func_owner(nested_), nested_func(nested_func_owner.get()) {}
|
||||
|
||||
String getName() const
|
||||
{
|
||||
return nested_func->getName() + "Array";
|
||||
}
|
||||
|
||||
DataTypePtr getReturnType() const
|
||||
{
|
||||
return nested_func->getReturnType();
|
||||
}
|
||||
|
||||
void setArguments(const DataTypes & arguments)
|
||||
{
|
||||
num_agruments = arguments.size();
|
||||
|
||||
DataTypes nested_arguments;
|
||||
for (int i = 0; i < num_agruments; ++i)
|
||||
{
|
||||
if (const DataTypeArray * array = dynamic_cast<const DataTypeArray *>(&*arguments[i]))
|
||||
nested_arguments.push_back(array->getNestedType());
|
||||
else
|
||||
throw Exception("Illegal type " + arguments[i]->getName() + " of argument #" + toString(i + 1) + " for aggregate function " + getName() + ". Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
nested_func->setArguments(nested_arguments);
|
||||
}
|
||||
|
||||
void setParameters(const Array & params)
|
||||
{
|
||||
nested_func->setParameters(params);
|
||||
}
|
||||
|
||||
void create(AggregateDataPtr place) const
|
||||
{
|
||||
nested_func->create(place);
|
||||
}
|
||||
|
||||
void destroy(AggregateDataPtr place) const
|
||||
{
|
||||
nested_func->destroy(place);
|
||||
}
|
||||
|
||||
bool hasTrivialDestructor() const
|
||||
{
|
||||
return nested_func->hasTrivialDestructor();
|
||||
}
|
||||
|
||||
size_t sizeOfData() const
|
||||
{
|
||||
return nested_func->sizeOfData();
|
||||
}
|
||||
|
||||
size_t alignOfData() const
|
||||
{
|
||||
return nested_func->alignOfData();
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
|
||||
{
|
||||
const IColumn * nested[num_agruments];
|
||||
|
||||
for (int i = 0; i < num_agruments; ++i)
|
||||
nested[i] = &static_cast<const ColumnArray &>(*columns[i]).getData();
|
||||
|
||||
const ColumnArray & first_array_column = static_cast<const ColumnArray &>(*columns[0]);
|
||||
const IColumn::Offsets_t & offsets = first_array_column.getOffsets();
|
||||
|
||||
size_t begin = row_num == 0 ? 0 : offsets[row_num - 1];
|
||||
size_t end = offsets[row_num];
|
||||
|
||||
for (size_t i = begin; i < end; ++i)
|
||||
nested_func->add(place, nested, i);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
|
||||
{
|
||||
nested_func->merge(place, rhs);
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
|
||||
{
|
||||
nested_func->serialize(place, buf);
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
|
||||
{
|
||||
nested_func->deserializeMerge(place, buf);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
|
||||
{
|
||||
nested_func->insertResultInto(place, to);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -47,7 +47,7 @@ public:
|
||||
nested_func->setArguments(nested_arguments);
|
||||
}
|
||||
|
||||
void setParameters(const Row & params)
|
||||
void setParameters(const Array & params)
|
||||
{
|
||||
nested_func->setParameters(params);
|
||||
}
|
||||
|
@ -56,7 +56,7 @@ public:
|
||||
type = argument;
|
||||
}
|
||||
|
||||
void setParameters(const Row & params)
|
||||
void setParameters(const Array & params)
|
||||
{
|
||||
if (params.size() != 1)
|
||||
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
@ -130,7 +130,7 @@ public:
|
||||
type = argument;
|
||||
}
|
||||
|
||||
void setParameters(const Row & params)
|
||||
void setParameters(const Array & params)
|
||||
{
|
||||
if (params.empty())
|
||||
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
@ -324,11 +324,13 @@ private:
|
||||
|
||||
void toLarge()
|
||||
{
|
||||
large = new detail::QuantileTimingLarge;
|
||||
/// На время копирования данных из tiny, устанавливать значение large ещё нельзя (иначе оно перезатрёт часть данных).
|
||||
detail::QuantileTimingLarge * tmp_large = new detail::QuantileTimingLarge;
|
||||
|
||||
for (size_t i = 0; i < tiny.count; ++i)
|
||||
large->insert(tiny.elems[i]);
|
||||
tmp_large->insert(tiny.elems[i]);
|
||||
|
||||
large = tmp_large;
|
||||
tiny.count = TINY_MAX_ELEMS + 1;
|
||||
}
|
||||
|
||||
@ -511,7 +513,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
void setParameters(const Row & params)
|
||||
void setParameters(const Array & params)
|
||||
{
|
||||
if (params.size() != 1)
|
||||
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
@ -570,7 +572,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
void setParameters(const Row & params)
|
||||
void setParameters(const Array & params)
|
||||
{
|
||||
if (params.empty())
|
||||
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
@ -39,9 +39,10 @@ public:
|
||||
* Если параметры не предусмотрены или переданные параметры недопустимы - кинуть исключение.
|
||||
* Если параметры есть - необходимо вызывать перед остальными вызовами, иначе - не вызывать.
|
||||
*/
|
||||
virtual void setParameters(const Row & params)
|
||||
virtual void setParameters(const Array & params)
|
||||
{
|
||||
throw Exception("Aggregate function " + getName() + " doesn't allow parameters.", ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS);
|
||||
throw Exception("Aggregate function " + getName() + " doesn't allow parameters.",
|
||||
ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS);
|
||||
}
|
||||
|
||||
/// Получить тип результата.
|
||||
|
@ -4,8 +4,8 @@
|
||||
|
||||
#include <Poco/Net/StreamSocket.h>
|
||||
|
||||
#include <DB/Core/Defines.h>
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/Core/Defines.h>
|
||||
#include <DB/Core/Progress.h>
|
||||
#include <DB/Core/Protocol.h>
|
||||
#include <DB/Core/QueryProcessingStage.h>
|
||||
@ -24,6 +24,10 @@ namespace DB
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
/// Поток блоков читающих из таблицы и ее имя
|
||||
typedef std::pair<BlockInputStreamPtr, std::string> ExternalTableData;
|
||||
/// Вектор пар, описывающих таблицы
|
||||
typedef std::vector<ExternalTableData> ExternalTablesData;
|
||||
|
||||
/** Соединение с сервером БД для использования в клиенте.
|
||||
* Как использовать - см. Core/Protocol.h
|
||||
@ -74,16 +78,23 @@ public:
|
||||
Packet() : type(Protocol::Server::Hello) {}
|
||||
};
|
||||
|
||||
/// Изменить базу данных по умолчанию. Изменения начинают использоваться только при следующем переподключении.
|
||||
void setDefaultDatabase(const String & database);
|
||||
|
||||
void getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & revision);
|
||||
|
||||
/// Адрес сервера - для сообщений в логе и в эксепшенах.
|
||||
String getServerAddress() const;
|
||||
|
||||
/// Если последний флаг true, то затем необходимо вызвать sendExternalTablesData
|
||||
void sendQuery(const String & query, const String & query_id_ = "", UInt64 stage = QueryProcessingStage::Complete,
|
||||
const Settings * settings = NULL);
|
||||
const Settings * settings = nullptr, bool with_pending_data = false);
|
||||
|
||||
void sendCancel();
|
||||
void sendData(const Block & block);
|
||||
/// Отправить блок данных, на сервере сохранить во временную таблицу name
|
||||
void sendData(const Block & block, const String & name = "");
|
||||
/// Отправить все содержимое внешних таблиц
|
||||
void sendExternalTablesData(ExternalTablesData & data);
|
||||
|
||||
/// Проверить, если ли данные, которые можно прочитать.
|
||||
bool poll(size_t timeout_microseconds = 0);
|
||||
|
@ -87,7 +87,7 @@ class IConnectionPool : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
typedef detail::ConnectionPoolEntry Entry;
|
||||
virtual Entry get(Settings * settings = NULL) = 0;
|
||||
virtual Entry get(Settings * settings = nullptr) = 0;
|
||||
virtual ~IConnectionPool() {}
|
||||
};
|
||||
|
||||
@ -122,7 +122,7 @@ public:
|
||||
|
||||
|
||||
/** Выделяет соединение для работы. */
|
||||
Entry get(Settings * settings = NULL)
|
||||
Entry get(Settings * settings = nullptr)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
@ -136,7 +136,11 @@ public:
|
||||
return Entry(allocConnection());
|
||||
|
||||
LOG_INFO(log, "No free connections in pool. Waiting.");
|
||||
available.wait(mutex);
|
||||
|
||||
if (settings)
|
||||
available.wait(mutex, settings->queue_max_wait_ms.totalMilliseconds());
|
||||
else
|
||||
available.wait(mutex);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -5,8 +5,6 @@
|
||||
|
||||
#include <DB/Client/ConnectionPool.h>
|
||||
|
||||
#include <boost/bind.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -28,7 +26,7 @@ public:
|
||||
ConnectionPoolWithFailover(ConnectionPools & nested_pools_,
|
||||
LoadBalancing load_balancing,
|
||||
size_t max_tries_ = DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES,
|
||||
size_t decrease_error_period_ = DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD)
|
||||
time_t decrease_error_period_ = DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD)
|
||||
: nested_pools(nested_pools_.begin(), nested_pools_.end(), decrease_error_period_), max_tries(max_tries_),
|
||||
log(&Logger::get("ConnectionPoolWithFailover")), default_load_balancing(load_balancing)
|
||||
{
|
||||
@ -41,22 +39,33 @@ public:
|
||||
if (settings)
|
||||
load_balancing = settings->load_balancing;
|
||||
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
/// Обновление случайных чисел, а также счётчиков ошибок.
|
||||
nested_pools.update(load_balancing);
|
||||
std::sort(nested_pools.begin(), nested_pools.end(), boost::bind(&PoolWithErrorCount::compare, _1, _2, load_balancing));
|
||||
|
||||
typedef std::vector<PoolWithErrorCount*> PoolPtrs;
|
||||
|
||||
size_t pools_size = nested_pools.size();
|
||||
PoolPtrs pool_ptrs(pools_size);
|
||||
for (size_t i = 0; i < pools_size; ++i)
|
||||
pool_ptrs[i] = &nested_pools[i];
|
||||
|
||||
std::sort(pool_ptrs.begin(), pool_ptrs.end(),
|
||||
[=](const PoolPtrs::value_type & lhs, const PoolPtrs::value_type & rhs)
|
||||
{
|
||||
return PoolWithErrorCount::compare(*lhs, *rhs, load_balancing);
|
||||
});
|
||||
|
||||
std::stringstream fail_messages;
|
||||
|
||||
for (size_t try_no = 0; try_no < max_tries; ++try_no)
|
||||
{
|
||||
for (size_t i = 0, size = nested_pools.size(); i < size; ++i)
|
||||
for (size_t i = 0; i < pools_size; ++i)
|
||||
{
|
||||
std::stringstream fail_message;
|
||||
|
||||
try
|
||||
{
|
||||
Entry res = nested_pools[i].pool->get();
|
||||
Entry res = pool_ptrs[i]->pool->get(settings);
|
||||
res->forceConnected();
|
||||
return res;
|
||||
}
|
||||
@ -73,8 +82,8 @@ public:
|
||||
|
||||
fail_messages << fail_message.str() << std::endl;
|
||||
|
||||
++nested_pools[i].random_error_count;
|
||||
++nested_pools[i].nearest_hostname_error_count;
|
||||
__sync_fetch_and_add(&pool_ptrs[i]->random_error_count, 1);
|
||||
__sync_fetch_and_add(&pool_ptrs[i]->nearest_hostname_error_count, 1);
|
||||
}
|
||||
}
|
||||
|
||||
@ -82,26 +91,26 @@ public:
|
||||
ErrorCodes::ALL_CONNECTION_TRIES_FAILED);
|
||||
}
|
||||
|
||||
|
||||
private:
|
||||
struct PoolWithErrorCount
|
||||
{
|
||||
ConnectionPoolPtr pool;
|
||||
|
||||
UInt64 random_error_count;
|
||||
UInt32 random;
|
||||
UInt64 random_error_count = 0;
|
||||
UInt32 random = 0;
|
||||
drand48_data rand_state;
|
||||
|
||||
/// берётся имя локального сервера (Poco::Net::DNS::hostName) и имя хоста из конфига; строки обрезаются до минимальной длины;
|
||||
/// затем считается количество отличающихся позиций
|
||||
/// Пример example01-01-1 и example01-02-2 отличаются в двух позициях.
|
||||
size_t hostname_difference;
|
||||
UInt64 nearest_hostname_error_count;
|
||||
size_t hostname_difference = 0;
|
||||
UInt64 nearest_hostname_error_count = 0;
|
||||
|
||||
PoolWithErrorCount(const ConnectionPoolPtr & pool_)
|
||||
: pool(pool_), random_error_count(0), random(0), nearest_hostname_error_count(0)
|
||||
PoolWithErrorCount(const ConnectionPoolPtr & pool_) : pool(pool_)
|
||||
{
|
||||
/// Инициализация плохая, но это не важно.
|
||||
srand48_r(reinterpret_cast<ptrdiff_t>(this), &rand_state);
|
||||
srand48_r(reinterpret_cast<intptr_t>(this), &rand_state);
|
||||
|
||||
std::string local_hostname = Poco::Net::DNS::hostName();
|
||||
|
||||
@ -109,12 +118,10 @@ private:
|
||||
const std::string & host = connection_pool.getHost();
|
||||
hostname_difference = 0;
|
||||
for (size_t i = 0; i < std::min(local_hostname.length(), host.length()); ++i)
|
||||
{
|
||||
if (local_hostname[i] != host[i])
|
||||
++hostname_difference;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void randomize()
|
||||
{
|
||||
long int rand_res;
|
||||
@ -126,64 +133,93 @@ private:
|
||||
{
|
||||
if (load_balancing_mode == LoadBalancing::RANDOM)
|
||||
{
|
||||
return lhs.random_error_count < rhs.random_error_count
|
||||
|| (lhs.random_error_count == rhs.random_error_count && lhs.random < rhs.random);
|
||||
return std::tie(lhs.random_error_count, lhs.random)
|
||||
< std::tie(rhs.random_error_count, rhs.random);
|
||||
}
|
||||
else if (load_balancing_mode == LoadBalancing::NEAREST_HOSTNAME)
|
||||
{
|
||||
return lhs.nearest_hostname_error_count < rhs.nearest_hostname_error_count
|
||||
|| (lhs.nearest_hostname_error_count == rhs.nearest_hostname_error_count
|
||||
&& lhs.hostname_difference < rhs.hostname_difference);
|
||||
return std::tie(lhs.nearest_hostname_error_count, lhs.hostname_difference)
|
||||
< std::tie(rhs.nearest_hostname_error_count, rhs.hostname_difference);
|
||||
}
|
||||
else
|
||||
throw Poco::Exception("Unsupported load_balancing_mode: " + toString(static_cast<int>(load_balancing_mode)));
|
||||
throw Exception("Unknown load_balancing_mode: " + toString(static_cast<int>(load_balancing_mode)), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class PoolsWithErrorCount : public std::vector<PoolWithErrorCount>
|
||||
{
|
||||
public:
|
||||
PoolsWithErrorCount(DB::ConnectionPools::iterator first, DB::ConnectionPools::iterator last,
|
||||
size_t decrease_error_period_) :
|
||||
std::vector<PoolWithErrorCount>(first, last), last_get_time(0),
|
||||
decrease_error_period(decrease_error_period_)
|
||||
PoolsWithErrorCount(DB::ConnectionPools::iterator begin_, DB::ConnectionPools::iterator end_,
|
||||
time_t decrease_error_period_)
|
||||
: std::vector<PoolWithErrorCount>(begin_, end_),
|
||||
decrease_error_period(decrease_error_period_)
|
||||
{
|
||||
}
|
||||
|
||||
void update(LoadBalancing load_balancing_mode)
|
||||
{
|
||||
if (load_balancing_mode == LoadBalancing::RANDOM)
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
switch (load_balancing_mode)
|
||||
{
|
||||
for (PoolsWithErrorCount::iterator it = begin(); it != end(); ++it)
|
||||
it->randomize();
|
||||
}
|
||||
/// Для режима NEAREST_HOSTNAME каждые N секунд уменьшаем количество ошибок в 2 раза
|
||||
if (last_get_time)
|
||||
{
|
||||
time_t delta = time(0) - last_get_time;
|
||||
for (PoolsWithErrorCount::iterator it = begin(); it != end(); ++it)
|
||||
case LoadBalancing::RANDOM:
|
||||
{
|
||||
it->nearest_hostname_error_count = it->nearest_hostname_error_count >> (delta / decrease_error_period);
|
||||
for (PoolsWithErrorCount::iterator it = begin(); it != end(); ++it)
|
||||
it->randomize();
|
||||
/// NOTE Почему бы не делить счётчики ошибок в случае LoadBalancing::RANDOM тоже?
|
||||
break;
|
||||
}
|
||||
|
||||
case LoadBalancing::NEAREST_HOSTNAME:
|
||||
{
|
||||
/// Для режима NEAREST_HOSTNAME каждые N секунд уменьшаем количество ошибок в 2 раза
|
||||
time_t current_time = time(0);
|
||||
|
||||
if (last_decrease_time)
|
||||
{
|
||||
time_t delta = current_time - last_decrease_time;
|
||||
|
||||
if (delta < 0)
|
||||
return;
|
||||
|
||||
/// Каждые decrease_error_period секунд, делим количество ошибок на два.
|
||||
size_t shift_amount = delta / decrease_error_period;
|
||||
|
||||
if (shift_amount > sizeof(UInt64))
|
||||
{
|
||||
last_decrease_time = current_time;
|
||||
for (PoolsWithErrorCount::iterator it = begin(); it != end(); ++it)
|
||||
it->nearest_hostname_error_count = 0;
|
||||
}
|
||||
else if (shift_amount)
|
||||
{
|
||||
last_decrease_time = current_time;
|
||||
for (PoolsWithErrorCount::iterator it = begin(); it != end(); ++it)
|
||||
it->nearest_hostname_error_count >>= shift_amount;
|
||||
}
|
||||
}
|
||||
else
|
||||
last_decrease_time = current_time;
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
throw Exception("Unknown load_balancing_mode: " + toString(static_cast<int>(load_balancing_mode)), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
last_get_time = time(0);
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
/// время, когда последний раз вызывался update
|
||||
time_t last_get_time;
|
||||
/// Время, когда последний раз уменьшался счётчик ошибок для LoadBalancing::NEAREST_HOSTNAME
|
||||
time_t last_decrease_time = 0;
|
||||
time_t decrease_error_period;
|
||||
Poco::FastMutex mutex;
|
||||
};
|
||||
|
||||
Poco::FastMutex mutex;
|
||||
|
||||
PoolsWithErrorCount nested_pools;
|
||||
|
||||
size_t max_tries;
|
||||
|
||||
Logger * log;
|
||||
|
||||
LoadBalancing default_load_balancing;
|
||||
};
|
||||
|
||||
|
@ -39,10 +39,8 @@ public:
|
||||
func = func_;
|
||||
}
|
||||
|
||||
AggregateFunctionPtr getAggregateFunction()
|
||||
{
|
||||
return func;
|
||||
}
|
||||
AggregateFunctionPtr getAggregateFunction() { return func; }
|
||||
AggregateFunctionPtr getAggregateFunction() const { return func; }
|
||||
|
||||
/// Захватить владение ареной.
|
||||
void addArena(ArenaPtr arena_)
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
typedef ColumnVector<Offset_t> ColumnOffsets_t;
|
||||
|
||||
/** Создать пустой столбец массивов, с типом значений, как в столбце nested_column */
|
||||
explicit ColumnArray(ColumnPtr nested_column, ColumnPtr offsets_column = NULL)
|
||||
explicit ColumnArray(ColumnPtr nested_column, ColumnPtr offsets_column = nullptr)
|
||||
: data(nested_column), offsets(offsets_column)
|
||||
{
|
||||
if (!offsets_column)
|
||||
@ -242,7 +242,7 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const final
|
||||
{
|
||||
const ColumnArray & rhs = static_cast<const ColumnArray &>(rhs_);
|
||||
|
||||
@ -251,7 +251,7 @@ public:
|
||||
size_t rhs_size = rhs.sizeAt(m);
|
||||
size_t min_size = std::min(lhs_size, rhs_size);
|
||||
for (size_t i = 0; i < min_size; ++i)
|
||||
if (int res = data->compareAt(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data, nan_direction_hint))
|
||||
if (int res = data.get()->compareAt(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data.get(), nan_direction_hint))
|
||||
return res;
|
||||
|
||||
return lhs_size < rhs_size
|
||||
@ -265,50 +265,41 @@ public:
|
||||
struct less
|
||||
{
|
||||
const ColumnArray & parent;
|
||||
const Permutation & nested_perm;
|
||||
|
||||
less(const ColumnArray & parent_, const Permutation & nested_perm_) : parent(parent_), nested_perm(nested_perm_) {}
|
||||
less(const ColumnArray & parent_) : parent(parent_) {}
|
||||
|
||||
bool operator()(size_t lhs, size_t rhs) const
|
||||
{
|
||||
size_t lhs_size = parent.sizeAt(lhs);
|
||||
size_t rhs_size = parent.sizeAt(rhs);
|
||||
size_t min_size = std::min(lhs_size, rhs_size);
|
||||
for (size_t i = 0; i < min_size; ++i)
|
||||
{
|
||||
if (nested_perm[parent.offsetAt(lhs) + i] < nested_perm[parent.offsetAt(rhs) + i])
|
||||
return positive;
|
||||
else if (nested_perm[parent.offsetAt(lhs) + i] > nested_perm[parent.offsetAt(rhs) + i])
|
||||
return !positive;
|
||||
}
|
||||
return positive == (lhs_size < rhs_size);
|
||||
if (positive)
|
||||
return parent.compareAt(lhs, rhs, parent, 1) < 0;
|
||||
else
|
||||
return parent.compareAt(lhs, rhs, parent, -1) > 0;
|
||||
}
|
||||
};
|
||||
|
||||
void getPermutation(bool reverse, size_t limit, Permutation & res) const
|
||||
{
|
||||
Permutation nested_perm;
|
||||
data->getPermutation(reverse, limit, nested_perm);
|
||||
size_t s = size();
|
||||
if (limit > s)
|
||||
limit = 0;
|
||||
|
||||
res.resize(s);
|
||||
for (size_t i = 0; i < s; ++i)
|
||||
res[i] = i;
|
||||
|
||||
if (limit > s)
|
||||
limit = 0;
|
||||
|
||||
if (limit)
|
||||
{
|
||||
if (reverse)
|
||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<false>(*this, nested_perm));
|
||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<false>(*this));
|
||||
else
|
||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<true>(*this, nested_perm));
|
||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<true>(*this));
|
||||
}
|
||||
else
|
||||
{
|
||||
if (reverse)
|
||||
std::sort(res.begin(), res.end(), less<false>(*this, nested_perm));
|
||||
std::sort(res.begin(), res.end(), less<false>(*this));
|
||||
else
|
||||
std::sort(res.begin(), res.end(), less<true>(*this, nested_perm));
|
||||
std::sort(res.begin(), res.end(), less<true>(*this));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
typedef ColumnVector<Offset_t> ColumnOffsets_t;
|
||||
|
||||
/** Создать пустой столбец вложенных таблиц, с типом значений, как в столбце nested_column */
|
||||
explicit ColumnNested(Columns nested_columns, ColumnPtr offsets_column = NULL)
|
||||
explicit ColumnNested(Columns nested_columns, ColumnPtr offsets_column = nullptr)
|
||||
: data(nested_columns), offsets(offsets_column)
|
||||
{
|
||||
if (!offsets_column)
|
||||
|
@ -92,7 +92,7 @@ private:
|
||||
public:
|
||||
Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2, size_t linear_growth_threshold_ = 128 * 1024 * 1024)
|
||||
: growth_factor(growth_factor_), linear_growth_threshold(linear_growth_threshold_),
|
||||
head(new Chunk(initial_size_, NULL)), size_in_bytes(head->size())
|
||||
head(new Chunk(initial_size_, nullptr)), size_in_bytes(head->size())
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -20,7 +20,7 @@ namespace DB
|
||||
* sizeof равен размеру одного указателя.
|
||||
*
|
||||
* Не exception-safe.
|
||||
* Копирование и присваивание разрушающее: исходный объект становится пустым.
|
||||
* Копирование не поддерживается. Перемещение опустошает исходный объект.
|
||||
* То есть, использовать этот массив во многих случаях неудобно.
|
||||
*
|
||||
* Предназначен для ситуаций, в которых создаётся много массивов одинакового небольшого размера,
|
||||
@ -82,24 +82,24 @@ public:
|
||||
init(size_, dont_init_elems);
|
||||
}
|
||||
|
||||
/** Разрушающее копирование.
|
||||
/** Премещение.
|
||||
*/
|
||||
AutoArray(const AutoArray & src)
|
||||
AutoArray(AutoArray && src)
|
||||
{
|
||||
//std::cerr << this << " AutoArray(const AutoArray & src)" << std::endl;
|
||||
|
||||
if (this == &src)
|
||||
return;
|
||||
setEmpty();
|
||||
data = src.data;
|
||||
const_cast<AutoArray<T> &>(src).setEmpty();
|
||||
src.setEmpty();
|
||||
}
|
||||
|
||||
AutoArray & operator= (const AutoArray & src)
|
||||
AutoArray & operator= (AutoArray && src)
|
||||
{
|
||||
//std::cerr << this << " operator=(const AutoArray & src)" << std::endl;
|
||||
|
||||
if (this == &src)
|
||||
return *this;
|
||||
uninit();
|
||||
data = src.data;
|
||||
const_cast<AutoArray<T> &>(src).setEmpty();
|
||||
src.setEmpty();
|
||||
|
||||
return *this;
|
||||
}
|
||||
|
212
dbms/include/DB/Common/ExternalTable.h
Normal file
212
dbms/include/DB/Common/ExternalTable.h
Normal file
@ -0,0 +1,212 @@
|
||||
#pragma once
|
||||
|
||||
#include <boost/program_options.hpp>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <DB/DataStreams/AsynchronousBlockInputStream.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/IO/copyData.h>
|
||||
#include <DB/IO/ReadBufferFromIStream.h>
|
||||
#include <DB/Storages/StorageMemory.h>
|
||||
#include <Poco/Net/HTMLForm.h>
|
||||
#include <Poco/Net/PartHandler.h>
|
||||
#include <Poco/Net/MessageHeader.h>
|
||||
#include <statdaemons/HTMLForm.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Базовый класс содержащий основную информацию о внешней таблице и
|
||||
/// основные функции для извлечения этой информации из текстовых полей.
|
||||
class BaseExternalTable
|
||||
{
|
||||
public:
|
||||
std::string file; /// Файл с данными или '-' если stdin
|
||||
std::string name; /// Имя таблицы
|
||||
std::string format; /// Название формата хранения данных
|
||||
|
||||
/// Описание структуры таблицы: (имя столбца, имя типа данных)
|
||||
std::vector<std::pair<std::string, std::string> > structure;
|
||||
|
||||
std::unique_ptr<ReadBuffer> read_buffer;
|
||||
Block sample_block;
|
||||
|
||||
virtual ~BaseExternalTable() {};
|
||||
|
||||
/// Инициализировать read_buffer в зависимости от источника данных. По умолчанию не делает ничего.
|
||||
virtual void initReadBuffer() {};
|
||||
|
||||
/// Инициализировать sample_block по структуре таблицы сохраненной в structure
|
||||
virtual void initSampleBlock(const Context & context)
|
||||
{
|
||||
for (size_t i = 0; i < structure.size(); ++i)
|
||||
{
|
||||
ColumnWithNameAndType column;
|
||||
column.name = structure[i].first;
|
||||
column.type = context.getDataTypeFactory().get(structure[i].second);
|
||||
column.column = column.type->createColumn();
|
||||
sample_block.insert(column);
|
||||
}
|
||||
}
|
||||
|
||||
/// Получить данные таблицы - пару (поток с содержимым таблицы, имя таблицы)
|
||||
virtual ExternalTableData getData(const Context & context)
|
||||
{
|
||||
initReadBuffer();
|
||||
initSampleBlock(context);
|
||||
ExternalTableData res = std::make_pair(new AsynchronousBlockInputStream(context.getFormatFactory().getInput(
|
||||
format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE, context.getDataTypeFactory())), name);
|
||||
return res;
|
||||
}
|
||||
|
||||
protected:
|
||||
/// Очистить всю накопленную информацию
|
||||
void clean()
|
||||
{
|
||||
name = "";
|
||||
file = "";
|
||||
format = "";
|
||||
structure.clear();
|
||||
sample_block = Block();
|
||||
read_buffer.reset();
|
||||
}
|
||||
|
||||
/// Функция для отладочного вывода информации
|
||||
virtual void write()
|
||||
{
|
||||
std::cerr << "file " << file << std::endl;
|
||||
std::cerr << "name " << name << std::endl;
|
||||
std::cerr << "format " << format << std::endl;
|
||||
std::cerr << "structure: \n";
|
||||
for (size_t i = 0; i < structure.size(); ++i)
|
||||
std::cerr << "\t" << structure[i].first << " " << structure[i].second << std::endl;
|
||||
}
|
||||
|
||||
static std::vector<std::string> split(const std::string & s, const std::string & d)
|
||||
{
|
||||
std::vector<std::string> res;
|
||||
boost::split(res, s, boost::algorithm::is_any_of(d), boost::algorithm::token_compress_on);
|
||||
return res;
|
||||
}
|
||||
|
||||
/// Построить вектор structure по текстовому полю structure
|
||||
virtual void parseStructureFromStructureField(const std::string & argument)
|
||||
{
|
||||
std::vector<std::string> vals = split(argument, " ,");
|
||||
|
||||
if (vals.size() & 1)
|
||||
throw Exception("Odd number of attributes in section structure", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
for (size_t i = 0; i < vals.size(); i += 2)
|
||||
structure.emplace_back(vals[i], vals[i + 1]);
|
||||
}
|
||||
|
||||
/// Построить вектор structure по текстовому полю types
|
||||
virtual void parseStructureFromTypesField(const std::string & argument)
|
||||
{
|
||||
std::vector<std::string> vals = split(argument, " ,");
|
||||
|
||||
for (size_t i = 0; i < vals.size(); ++i)
|
||||
structure.emplace_back("_" + toString(i + 1), vals[i]);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Парсинг внешей таблицы, используемый в tcp клиенте.
|
||||
class ExternalTable : public BaseExternalTable
|
||||
{
|
||||
public:
|
||||
void initReadBuffer()
|
||||
{
|
||||
if (file == "-")
|
||||
read_buffer.reset(new ReadBufferFromFileDescriptor(STDIN_FILENO));
|
||||
else
|
||||
read_buffer.reset(new ReadBufferFromFile(file));
|
||||
}
|
||||
|
||||
/// Извлечение параметров из variables_map, которая строится по командной строке клиента
|
||||
ExternalTable(const boost::program_options::variables_map & external_options)
|
||||
{
|
||||
if (external_options.count("file"))
|
||||
file = external_options["file"].as<std::string>();
|
||||
else
|
||||
throw Exception("--file field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (external_options.count("name"))
|
||||
name = external_options["name"].as<std::string>();
|
||||
else
|
||||
throw Exception("--name field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (external_options.count("format"))
|
||||
format = external_options["format"].as<std::string>();
|
||||
else
|
||||
throw Exception("--format field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (external_options.count("structure"))
|
||||
parseStructureFromStructureField(external_options["structure"].as<std::string>());
|
||||
else if (external_options.count("types"))
|
||||
parseStructureFromTypesField(external_options["types"].as<std::string>());
|
||||
else
|
||||
throw Exception("Neither --structure nor --types have not been provided for external table", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
};
|
||||
|
||||
/// Парсинг внешей таблицы, используемый при отправке таблиц через http
|
||||
/// Функция handlePart будет вызываться для каждой переданной таблицы,
|
||||
/// поэтому так же необходимо вызывать clean в конце handlePart.
|
||||
class ExternalTablesHandler : public Poco::Net::PartHandler, BaseExternalTable
|
||||
{
|
||||
public:
|
||||
std::vector<std::string> names;
|
||||
|
||||
ExternalTablesHandler(Context & context_, Poco::Net::NameValueCollection params_) : context(context_), params(params_) { }
|
||||
|
||||
void handlePart(const Poco::Net::MessageHeader& header, std::istream& stream)
|
||||
{
|
||||
/// Буфер инициализируется здесь, а не в виртуальной функции initReadBuffer
|
||||
read_buffer.reset(new ReadBufferFromIStream(stream));
|
||||
|
||||
/// Извлекаем коллекцию параметров из MessageHeader
|
||||
Poco::Net::NameValueCollection content;
|
||||
std::string label;
|
||||
Poco::Net::MessageHeader::splitParameters(header.get("Content-Disposition"), label, content);
|
||||
|
||||
/// Получаем параметры
|
||||
name = content.get("name", "_data");
|
||||
format = params.get(name + "_format", "TabSeparated");
|
||||
|
||||
if (params.has(name + "_structure"))
|
||||
parseStructureFromStructureField(params.get(name + "_structure"));
|
||||
else if (params.has(name + "_types"))
|
||||
parseStructureFromTypesField(params.get(name + "_types"));
|
||||
else
|
||||
throw Exception("Neither structure nor types have not been provided for external table " + name + ". Use fields " + name + "_structure or " + name + "_types to do so.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
ExternalTableData data = getData(context);
|
||||
|
||||
/// Создаем таблицу
|
||||
NamesAndTypesListPtr columns = new NamesAndTypesList(sample_block.getColumnsList());
|
||||
StoragePtr storage = StorageMemory::create(data.second, columns);
|
||||
context.addExternalTable(data.second, storage);
|
||||
BlockOutputStreamPtr output = storage->write(ASTPtr());
|
||||
|
||||
/// Записываем данные
|
||||
data.first->readPrefix();
|
||||
output->writePrefix();
|
||||
while(Block block = data.first->read())
|
||||
output->write(block);
|
||||
data.first->readSuffix();
|
||||
output->writeSuffix();
|
||||
|
||||
names.push_back(name);
|
||||
/// Подготавливаемся к приему следующего файла, для этого очищаем всю полученную информацию
|
||||
clean();
|
||||
}
|
||||
|
||||
private:
|
||||
Context & context;
|
||||
Poco::Net::NameValueCollection params;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -112,8 +112,13 @@ public:
|
||||
current_size = 0;
|
||||
hits = 0;
|
||||
misses = 0;
|
||||
current_weight_lost = 0;
|
||||
}
|
||||
|
||||
protected:
|
||||
size_t current_weight_lost = 0;
|
||||
/// Суммарный вес выброшенных из кеша элементов.
|
||||
/// Обнуляется каждый раз, когда информация добавляется в Profile events
|
||||
private:
|
||||
typedef std::list<Key> LRUQueue;
|
||||
typedef typename LRUQueue::iterator LRUQueueIterator;
|
||||
@ -148,6 +153,7 @@ private:
|
||||
const Key & key = queue.front();
|
||||
auto it = cells.find(key);
|
||||
current_size -= it->second.size;
|
||||
current_weight_lost += it->second.size;
|
||||
cells.erase(it);
|
||||
queue.pop_front();
|
||||
--queue_size;
|
||||
|
@ -83,20 +83,18 @@ private:
|
||||
{
|
||||
if (n == 0)
|
||||
{
|
||||
c_start = c_end = c_end_of_storage = NULL;
|
||||
c_start = c_end = c_end_of_storage = nullptr;
|
||||
return;
|
||||
}
|
||||
|
||||
size_t bytes_to_alloc = to_size(n);
|
||||
c_start = c_end = Allocator::allocate(bytes_to_alloc);
|
||||
c_end_of_storage = c_start + bytes_to_alloc;
|
||||
|
||||
//memset(c_start, 0, bytes_to_alloc);
|
||||
}
|
||||
|
||||
void dealloc()
|
||||
{
|
||||
if (c_start == NULL)
|
||||
if (c_start == nullptr)
|
||||
return;
|
||||
|
||||
if (use_libc_realloc)
|
||||
@ -107,9 +105,7 @@ private:
|
||||
|
||||
void realloc(size_t n)
|
||||
{
|
||||
// std::cerr << "realloc" << std::endl;
|
||||
|
||||
if (c_start == NULL)
|
||||
if (c_start == nullptr)
|
||||
{
|
||||
alloc(n);
|
||||
return;
|
||||
@ -125,22 +121,20 @@ private:
|
||||
{
|
||||
c_start = reinterpret_cast<char *>(::realloc(c_start, bytes_to_alloc));
|
||||
|
||||
if (NULL == c_start)
|
||||
if (nullptr == c_start)
|
||||
throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
}
|
||||
else
|
||||
{
|
||||
c_start = reinterpret_cast<char *>(malloc(bytes_to_alloc));
|
||||
|
||||
if (NULL == c_start)
|
||||
if (nullptr == c_start)
|
||||
throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
|
||||
memcpy(c_start, old_c_start, old_c_end_of_storage - old_c_start);
|
||||
Allocator::deallocate(old_c_start, old_c_end_of_storage - old_c_start);
|
||||
}
|
||||
|
||||
//memset(c_start + (old_c_end_of_storage - old_c_start), 0, bytes_to_alloc - (old_c_end_of_storage - old_c_start));
|
||||
|
||||
|
||||
c_end = c_start + end_diff;
|
||||
c_end_of_storage = c_start + bytes_to_alloc;
|
||||
|
||||
|
@ -19,6 +19,7 @@
|
||||
M(CompressedReadBufferBytes, "Read decompressed bytes") \
|
||||
M(UncompressedCacheHits, "Uncompressed cache hits") \
|
||||
M(UncompressedCacheMisses, "Uncompressed cache misses") \
|
||||
M(UncompressedCacheWeightLost, "Uncompressed cache weight lost") \
|
||||
M(IOBufferAllocs, "IO buffers allocations") \
|
||||
M(IOBufferAllocBytes, "IO buffers allocated bytes") \
|
||||
M(ArenaAllocChunks, "Arena allocated chunks") \
|
||||
@ -27,6 +28,12 @@
|
||||
M(MarkCacheHits, "Mark cache hits") \
|
||||
M(MarkCacheMisses, "Mark cache misses") \
|
||||
\
|
||||
M(ReplicatedPartFetches, "Replicated part fetches") \
|
||||
M(ReplicatedPartFailedFetches, "Replicated part fetches failed") \
|
||||
M(ObsoleteReplicatedParts, "Replicated parts rendered obsolete by fetches") \
|
||||
M(ReplicatedPartMerges, "Replicated part merges") \
|
||||
M(ReplicatedPartFetchesOfMerged, "Replicated part merges replaced with fetches") \
|
||||
\
|
||||
M(END, "")
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <DB/Parsers/ASTExpressionList.h>
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
#include <DB/Storages/StoragePtr.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
|
||||
namespace DB
|
||||
@ -35,9 +34,9 @@ BlockInputStreamPtr getVirtualColumnsBlocks(ASTPtr query, const Block & input, c
|
||||
|
||||
/// Извлечь из входного потока множество значений столбца name
|
||||
template<typename T1>
|
||||
std::set<T1> extractSingleValueFromBlocks(BlockInputStreamPtr input, const String & name)
|
||||
std::multiset<T1> extractSingleValueFromBlocks(BlockInputStreamPtr input, const String & name)
|
||||
{
|
||||
std::set<T1> res;
|
||||
std::multiset<T1> res;
|
||||
input->readPrefix();
|
||||
while(1)
|
||||
{
|
||||
@ -50,28 +49,6 @@ std::set<T1> extractSingleValueFromBlocks(BlockInputStreamPtr input, const Strin
|
||||
return res;
|
||||
}
|
||||
|
||||
/// Извлечь из входного потока множество пар значений в столбцах first_name и second_name
|
||||
template<typename T1, typename T2>
|
||||
std::set< std::pair<T1, T2> > extractTwoValuesFromBlocks(BlockInputStreamPtr input,
|
||||
const String & first_name, const String & second_name)
|
||||
{
|
||||
std::set< std::pair<T1, T2> > res;
|
||||
input->readPrefix();
|
||||
while(1)
|
||||
{
|
||||
Block block = input->read();
|
||||
if (!block) break;
|
||||
const ColumnWithNameAndType & first = block.getByName(first_name);
|
||||
const ColumnWithNameAndType & second = block.getByName(second_name);
|
||||
for (size_t i = 0; i < block.rows(); ++i)
|
||||
{
|
||||
T1 val1 = (*first.column)[i].get<T1>();
|
||||
T2 val2 = (*second.column)[i].get<T2>();
|
||||
res.insert(std::make_pair(val1, val2));
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -1,5 +1,4 @@
|
||||
#ifndef DBMS_CORE_COLUMN_NUMBERS_H
|
||||
#define DBMS_CORE_COLUMN_NUMBERS_H
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <string>
|
||||
@ -11,5 +10,3 @@ namespace DB
|
||||
typedef std::vector<size_t> ColumnNumbers;
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -13,12 +13,19 @@
|
||||
#define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300
|
||||
#define DBMS_DEFAULT_POLL_INTERVAL 10
|
||||
|
||||
/// При записи данных, для сжатия выделяется буфер размером max_compress_block_size. При переполнении буфера или если в буфер
|
||||
/// записано данных больше или равно, чем min_compress_block_size, то при очередной засечке, данные так же будут сжиматься
|
||||
/// В результате, для маленьких столбцов (числа 1-8 байт), при index_granularity = 8192, размер блока будет 64 KБ.
|
||||
/// А для больших столбцов (Title - строка ~100 байт), размер блока будет ~819 КБ. За счёт этого, коэффициент сжатия почти не ухудшится.
|
||||
#define DEFAULT_MIN_COMPRESS_BLOCK_SIZE 65536
|
||||
#define DEFAULT_MAX_COMPRESS_BLOCK_SIZE 1048576
|
||||
|
||||
/// Какими блоками по-умолчанию читаются и пишутся данные (в числе строк).
|
||||
#define DEFAULT_BLOCK_SIZE 1048576
|
||||
/// То же самое, но для операций слияния. Меньше DEFAULT_BLOCK_SIZE для экономии оперативки (так как читаются все столбцы).
|
||||
#define DEFAULT_MERGE_BLOCK_SIZE 10000
|
||||
#define DEFAULT_MERGE_BLOCK_SIZE 8192
|
||||
|
||||
#define DEFAULT_MAX_QUERY_SIZE 1048576
|
||||
#define DEFAULT_MAX_QUERY_SIZE 65536
|
||||
#define SHOW_CHARS_ON_SYNTAX_ERROR 160L
|
||||
#define DEFAULT_MAX_THREADS 8
|
||||
#define DEFAULT_MAX_DISTRIBUTED_CONNECTIONS 1024
|
||||
@ -40,3 +47,4 @@
|
||||
#define DBMS_MIN_REVISION_WITH_USER_PASSWORD 34482
|
||||
#define DBMS_MIN_REVISION_WITH_TOTALS_EXTREMES 35265
|
||||
#define DBMS_MIN_REVISION_WITH_STRING_QUERY_ID 39002
|
||||
#define DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES 50264
|
||||
|
@ -225,6 +225,27 @@ namespace ErrorCodes
|
||||
NOT_AN_AGGREGATE,
|
||||
QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING,
|
||||
CLIENT_HAS_CONNECTED_TO_WRONG_PORT,
|
||||
TABLE_IS_DROPPED,
|
||||
DATABASE_NOT_EMPTY,
|
||||
DUPLICATE_INTERSERVER_IO_ENDPOINT,
|
||||
NO_SUCH_INTERSERVER_IO_ENDPOINT,
|
||||
ADDING_REPLICA_TO_NON_EMPTY_TABLE,
|
||||
UNEXPECTED_AST_STRUCTURE,
|
||||
REPLICA_IS_ALREADY_ACTIVE,
|
||||
NO_ZOOKEEPER,
|
||||
NO_FILE_IN_DATA_PART,
|
||||
UNEXPECTED_FILE_IN_DATA_PART,
|
||||
BAD_SIZE_OF_FILE_IN_DATA_PART,
|
||||
QUERY_IS_TOO_LARGE,
|
||||
NOT_FOUND_EXPECTED_DATA_PART,
|
||||
TOO_MANY_UNEXPECTED_DATA_PARTS,
|
||||
NO_SUCH_DATA_PART,
|
||||
BAD_DATA_PART_NAME,
|
||||
NO_REPLICA_HAS_PART,
|
||||
DUPLICATE_DATA_PART,
|
||||
ABORTED,
|
||||
NO_REPLICA_NAME_GIVEN,
|
||||
FORMAT_VERSION_TOO_OLD,
|
||||
|
||||
POCO_EXCEPTION = 1000,
|
||||
STD_EXCEPTION,
|
||||
|
@ -3,40 +3,13 @@
|
||||
#include <cerrno>
|
||||
#include <vector>
|
||||
|
||||
#include <Poco/Exception.h>
|
||||
#include <statdaemons/Exception.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/StackTrace.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Exception : public Poco::Exception
|
||||
{
|
||||
public:
|
||||
Exception(int code = 0);
|
||||
Exception(const std::string & msg, int code = 0);
|
||||
Exception(const std::string & msg, const std::string & arg, int code = 0);
|
||||
Exception(const std::string & msg, const Exception & exc, int code = 0);
|
||||
Exception(const Exception & exc);
|
||||
explicit Exception(const Poco::Exception & exc);
|
||||
~Exception() throw();
|
||||
Exception & operator = (const Exception & exc);
|
||||
const char * name() const throw();
|
||||
const char * className() const throw();
|
||||
Exception * clone() const;
|
||||
void rethrow() const;
|
||||
|
||||
/// Дописать к существующему сообщению что-нибудь ещё.
|
||||
void addMessage(const std::string & arg);
|
||||
|
||||
const StackTrace & getStackTrace() const { return trace; }
|
||||
|
||||
private:
|
||||
StackTrace trace;
|
||||
};
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
typedef SharedPtr<Poco::Exception> ExceptionPtr;
|
||||
|
@ -87,7 +87,6 @@ public:
|
||||
Field()
|
||||
: which(Types::Null)
|
||||
{
|
||||
// std::cerr << "Field()" << std::endl;
|
||||
}
|
||||
|
||||
/** Не смотря на наличие шаблонного конструктора, этот конструктор всё-равно нужен,
|
||||
@ -95,13 +94,11 @@ public:
|
||||
*/
|
||||
Field(const Field & rhs)
|
||||
{
|
||||
// std::cerr << this << " Field::Field(const Field &)" << std::endl;
|
||||
create(rhs);
|
||||
}
|
||||
|
||||
Field & operator= (const Field & rhs)
|
||||
{
|
||||
// std::cerr << this << " Field::operator=(const Field &)" << std::endl;
|
||||
destroy();
|
||||
create(rhs);
|
||||
return *this;
|
||||
@ -110,7 +107,6 @@ public:
|
||||
template <typename T>
|
||||
Field(const T & rhs)
|
||||
{
|
||||
// std::cerr << this << " Field::Field(" << Types::toString(TypeToEnum<T>::value) << ")" << std::endl;
|
||||
create(rhs);
|
||||
}
|
||||
|
||||
@ -140,7 +136,6 @@ public:
|
||||
template <typename T>
|
||||
Field & operator= (const T & rhs)
|
||||
{
|
||||
// std::cerr << this << " Field::operator=(" << Types::toString(TypeToEnum<T>::value) << ")" << std::endl;
|
||||
destroy();
|
||||
create(rhs);
|
||||
return *this;
|
||||
@ -148,7 +143,6 @@ public:
|
||||
|
||||
~Field()
|
||||
{
|
||||
// std::cerr << this << " Field::~Field()" << std::endl;
|
||||
destroy();
|
||||
}
|
||||
|
||||
@ -285,7 +279,6 @@ private:
|
||||
void create(const T & x)
|
||||
{
|
||||
which = TypeToEnum<T>::value;
|
||||
// std::cerr << this << " Creating " << getTypeName() << std::endl;
|
||||
T * __attribute__((__may_alias__)) ptr = reinterpret_cast<T*>(storage);
|
||||
new (ptr) T(x);
|
||||
}
|
||||
@ -293,13 +286,10 @@ private:
|
||||
void create(const Null & x)
|
||||
{
|
||||
which = Types::Null;
|
||||
// std::cerr << this << " Creating " << getTypeName() << std::endl;
|
||||
}
|
||||
|
||||
void create(const Field & x)
|
||||
{
|
||||
// std::cerr << this << " Creating Field" << std::endl;
|
||||
|
||||
switch (x.which)
|
||||
{
|
||||
case Types::Null: create(Null()); break;
|
||||
@ -326,8 +316,6 @@ private:
|
||||
|
||||
__attribute__((__always_inline__)) void destroy()
|
||||
{
|
||||
// std::cerr << this << " Destroying " << getTypeName() << std::endl;
|
||||
|
||||
if (which < Types::MIN_NON_POD)
|
||||
return;
|
||||
|
||||
@ -352,7 +340,36 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
static std::ostream & operator<< (std::ostream & os, const Field & x)
|
||||
{
|
||||
bool first = false;
|
||||
switch (x.getType())
|
||||
{
|
||||
case Field::Types::Null: os << "Null"; break;
|
||||
case Field::Types::UInt64: os << x.get<UInt64>(); break;
|
||||
case Field::Types::Int64: os << x.get<Int64>(); break;
|
||||
case Field::Types::Float64: os << x.get<Float64>(); break;
|
||||
case Field::Types::String: os << x.get<String>(); break;
|
||||
case Field::Types::Array:
|
||||
for(const Field & f : x.get<Array>())
|
||||
{
|
||||
os << "[";
|
||||
if (first)
|
||||
os << f;
|
||||
else
|
||||
os << ", " << f;
|
||||
os << "]";
|
||||
}
|
||||
break;
|
||||
default:
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << "Unsupported type " << x.getTypeName();
|
||||
throw DB::Exception(ss.str());
|
||||
}
|
||||
}
|
||||
return os;
|
||||
}
|
||||
template <> struct Field::TypeToEnum<Null> { static const Types::Which value = Types::Null; };
|
||||
template <> struct Field::TypeToEnum<UInt64> { static const Types::Which value = Types::UInt64; };
|
||||
template <> struct Field::TypeToEnum<Int64> { static const Types::Which value = Types::Int64; };
|
||||
|
@ -1,26 +0,0 @@
|
||||
#ifndef DBMS_CORE_NAME_AND_TYPE_H
|
||||
#define DBMS_CORE_NAME_AND_TYPE_H
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Column.h>
|
||||
#include <DB/DataTypes/IDataType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
/** Имя столбца и тип столбца.
|
||||
*/
|
||||
|
||||
struct NameAndType
|
||||
{
|
||||
DataTypePtr type;
|
||||
String name;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -17,6 +17,11 @@ namespace DB
|
||||
*
|
||||
* 1. Клиент отправляет на сервер пакет Query.
|
||||
*
|
||||
* Начиная с версии 50263, сразу после отправки пакета Query клиент начинает передачу
|
||||
* внешних (временных) таблиц (external storages) - один или несколько пакетов Data.
|
||||
* Конец передачи данных определается по отправленному пустому блоку.
|
||||
* В данный момент, не пустое множество таблиц может быть передано только вместе с запросом SELECT.
|
||||
*
|
||||
* Если запрос типа INSERT (требует передачи данных от клиента), то сервер передаёт
|
||||
* пакет Data, содержащий пустой блок, который описывает структуру таблицы.
|
||||
* Затем клиент отправляет данные для вставки
|
||||
|
@ -20,10 +20,10 @@ struct SortColumnDescription
|
||||
int direction; /// 1 - по возрастанию, -1 - по убыванию.
|
||||
Poco::SharedPtr<Collator> collator; /// Collator для locale-specific сортировки строк
|
||||
|
||||
SortColumnDescription(size_t column_number_, int direction_, const Poco::SharedPtr<Collator> & collator_ = NULL)
|
||||
SortColumnDescription(size_t column_number_, int direction_, const Poco::SharedPtr<Collator> & collator_ = nullptr)
|
||||
: column_number(column_number_), direction(direction_), collator(collator_) {}
|
||||
|
||||
SortColumnDescription(String column_name_, int direction_, const Poco::SharedPtr<Collator> & collator_ = NULL)
|
||||
SortColumnDescription(String column_name_, int direction_, const Poco::SharedPtr<Collator> & collator_ = nullptr)
|
||||
: column_name(column_name_), column_number(0), direction(direction_), collator(collator_) {}
|
||||
|
||||
/// Для IBlockInputStream.
|
||||
|
@ -1,28 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#define DBMS_STACK_TRACE_MAX_DEPTH 32
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Позволяет получить стек-трейс
|
||||
class StackTrace
|
||||
{
|
||||
public:
|
||||
/// Стектрейс снимается в момент создания объекта
|
||||
StackTrace();
|
||||
|
||||
/// Вывести в строку
|
||||
std::string toString() const;
|
||||
|
||||
private:
|
||||
typedef void* Frame;
|
||||
Frame frames[DBMS_STACK_TRACE_MAX_DEPTH];
|
||||
size_t frames_size;
|
||||
};
|
||||
|
||||
}
|
@ -7,6 +7,7 @@
|
||||
#include <vector>
|
||||
|
||||
#include <functional>
|
||||
#include <ostream>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -14,13 +15,13 @@ namespace DB
|
||||
/// Штука, чтобы не создавать строки для поиска подстроки в хэш таблице.
|
||||
struct StringRef
|
||||
{
|
||||
const char * data;
|
||||
size_t size;
|
||||
const char * data = nullptr;
|
||||
size_t size = 0;
|
||||
|
||||
StringRef(const char * data_, size_t size_) : data(data_), size(size_) {}
|
||||
StringRef(const unsigned char * data_, size_t size_) : data(reinterpret_cast<const char *>(data_)), size(size_) {}
|
||||
StringRef(const std::string & s) : data(s.data()), size(s.size()) {}
|
||||
StringRef() : data(NULL), size(0) {}
|
||||
StringRef() {}
|
||||
|
||||
std::string toString() const { return std::string(data, size); }
|
||||
};
|
||||
@ -46,6 +47,15 @@ namespace DB
|
||||
return !(lhs == rhs);
|
||||
}
|
||||
|
||||
inline bool operator<(StringRef lhs, StringRef rhs)
|
||||
{
|
||||
int cmp = memcmp(lhs.data, rhs.data, std::min(lhs.size, rhs.size));
|
||||
if (cmp == 0)
|
||||
return lhs.size < rhs.size;
|
||||
else
|
||||
return cmp < 0;
|
||||
}
|
||||
|
||||
struct StringRefHash
|
||||
{
|
||||
inline size_t operator() (StringRef x) const
|
||||
@ -56,8 +66,8 @@ namespace DB
|
||||
|
||||
struct StringRefZeroTraits
|
||||
{
|
||||
static inline bool check(DB::StringRef x) { return NULL == x.data; }
|
||||
static inline void set(DB::StringRef & x) { x.data = NULL; }
|
||||
static inline bool check(DB::StringRef x) { return nullptr == x.data; }
|
||||
static inline void set(DB::StringRef & x) { x.data = nullptr; }
|
||||
};
|
||||
|
||||
inline bool operator==(StringRef lhs, const char * rhs)
|
||||
@ -69,6 +79,14 @@ namespace DB
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
inline std::ostream & operator<<(std::ostream & os, const StringRef & str)
|
||||
{
|
||||
if (str.data)
|
||||
os.write(str.data, str.size);
|
||||
|
||||
return os;
|
||||
}
|
||||
}
|
||||
|
||||
namespace std
|
||||
|
@ -14,10 +14,10 @@ namespace DB
|
||||
class CollapsingFinalBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
CollapsingFinalBlockInputStream(BlockInputStreams inputs_, SortDescription & description_,
|
||||
CollapsingFinalBlockInputStream(BlockInputStreams inputs_, const SortDescription & description_,
|
||||
const String & sign_column_)
|
||||
: description(description_), sign_column(sign_column_),
|
||||
log(&Logger::get("CollapsingSortedBlockInputStream")),
|
||||
log(&Logger::get("CollapsingFinalBlockInputStream")),
|
||||
first(true), count_positive(0), count_negative(0), count_incorrect_data(0), blocks_fetched(0), blocks_output(0)
|
||||
{
|
||||
children.insert(children.end(), inputs_.begin(), inputs_.end());
|
||||
@ -144,7 +144,7 @@ private:
|
||||
--ptr->refcount;
|
||||
if (!ptr->refcount)
|
||||
delete ptr;
|
||||
ptr = NULL;
|
||||
ptr = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
@ -168,7 +168,7 @@ private:
|
||||
else
|
||||
ptr->output_blocks->push_back(ptr);
|
||||
}
|
||||
ptr = NULL;
|
||||
ptr = nullptr;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -23,7 +23,7 @@ namespace DB
|
||||
class CollapsingSortedBlockInputStream : public MergingSortedBlockInputStream
|
||||
{
|
||||
public:
|
||||
CollapsingSortedBlockInputStream(BlockInputStreams inputs_, SortDescription & description_,
|
||||
CollapsingSortedBlockInputStream(BlockInputStreams inputs_, const SortDescription & description_,
|
||||
const String & sign_column_, size_t max_block_size_)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_),
|
||||
sign_column(sign_column_), sign_column_number(0),
|
||||
|
@ -17,8 +17,10 @@ namespace DB
|
||||
class DistinctBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
DistinctBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_ = 0)
|
||||
: limit(limit_),
|
||||
/// Пустой columns_ значит все столбцы.
|
||||
DistinctBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_, Names columns_)
|
||||
: columns_names(columns_),
|
||||
limit(limit_),
|
||||
max_rows(limits.max_rows_in_distinct),
|
||||
max_bytes(limits.max_bytes_in_distinct),
|
||||
overflow_mode(limits.distinct_overflow_mode)
|
||||
@ -51,11 +53,17 @@ protected:
|
||||
return Block();
|
||||
|
||||
size_t rows = block.rows();
|
||||
size_t columns = block.columns();
|
||||
size_t columns = columns_names.empty() ? block.columns() : columns_names.size();
|
||||
|
||||
ConstColumnPlainPtrs column_ptrs(columns);
|
||||
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
column_ptrs[i] = block.getByPosition(i).column;
|
||||
{
|
||||
if (columns_names.empty())
|
||||
column_ptrs[i] = block.getByPosition(i).column;
|
||||
else
|
||||
column_ptrs[i] = block.getByName(columns_names[i]).column;
|
||||
}
|
||||
|
||||
/// Будем фильтровать блок, оставляя там только строки, которых мы ещё не видели.
|
||||
IColumn::Filter filter(rows);
|
||||
@ -111,7 +119,8 @@ protected:
|
||||
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
size_t all_columns = block.columns();
|
||||
for (size_t i = 0; i < all_columns; ++i)
|
||||
block.getByPosition(i).column = block.getByPosition(i).column->filter(filter);
|
||||
|
||||
return block;
|
||||
@ -129,6 +138,8 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
Names columns_names;
|
||||
|
||||
size_t limit;
|
||||
|
||||
/// Ограничения на максимальный размер множества
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/Storages/StoragePtr.h>
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -30,11 +30,7 @@ public:
|
||||
typedef SharedPtr<IBlockInputStream> BlockInputStreamPtr;
|
||||
typedef std::vector<BlockInputStreamPtr> BlockInputStreams;
|
||||
|
||||
/** Листовой BlockInputStream обычно требует, чтобы был жив какой-то Storage.
|
||||
* Переданный сюда указатель на Storage будет просто храниться в этом экземпляре,
|
||||
* не позволяя уничтожить Storage раньше этого BlockInputStream.
|
||||
*/
|
||||
IBlockInputStream(StoragePtr owned_storage_ = StoragePtr()) : owned_storage(owned_storage_) {}
|
||||
IBlockInputStream() {}
|
||||
|
||||
/** Прочитать следующий блок.
|
||||
* Если блоков больше нет - вернуть пустой блок (для которого operator bool возвращает false).
|
||||
@ -80,15 +76,17 @@ public:
|
||||
*/
|
||||
size_t checkDepth(size_t max_depth) const;
|
||||
|
||||
void setOwnedStorage(StoragePtr owned_storage_) { owned_storage = owned_storage_; }
|
||||
/** Не давать изменить таблицу, пока жив поток блоков.
|
||||
*/
|
||||
void addTableLock(const IStorage::TableStructureReadLockPtr & lock) { table_locks.push_back(lock); }
|
||||
|
||||
protected:
|
||||
StoragePtr owned_storage;
|
||||
IStorage::TableStructureReadLocks table_locks;
|
||||
|
||||
BlockInputStreams children;
|
||||
|
||||
private:
|
||||
void getLeavesImpl(BlockInputStreams & res, BlockInputStreamPtr this_shared_ptr = NULL);
|
||||
void getLeavesImpl(BlockInputStreams & res, BlockInputStreamPtr this_shared_ptr = nullptr);
|
||||
|
||||
size_t checkDepthImpl(size_t max_depth, size_t level) const;
|
||||
|
||||
@ -99,8 +97,5 @@ private:
|
||||
};
|
||||
|
||||
|
||||
typedef IBlockInputStream::BlockInputStreamPtr BlockInputStreamPtr;
|
||||
typedef IBlockInputStream::BlockInputStreams BlockInputStreams;
|
||||
|
||||
}
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/Core/Row.h>
|
||||
#include <DB/Storages/StoragePtr.h>
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -21,7 +21,7 @@ class IBlockOutputStream : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
|
||||
IBlockOutputStream(StoragePtr owned_storage_ = StoragePtr()) : owned_storage(owned_storage_) {}
|
||||
IBlockOutputStream() {}
|
||||
|
||||
/** Записать блок.
|
||||
*/
|
||||
@ -39,11 +39,13 @@ public:
|
||||
virtual void setExtremes(const Block & extremes) {}
|
||||
|
||||
virtual ~IBlockOutputStream() {}
|
||||
|
||||
/** Не давать изменить таблицу, пока жив поток блоков.
|
||||
*/
|
||||
void addTableLock(const IStorage::TableStructureReadLockPtr & lock) { table_locks.push_back(lock); }
|
||||
|
||||
protected:
|
||||
StoragePtr owned_storage;
|
||||
IStorage::TableStructureReadLocks table_locks;
|
||||
};
|
||||
|
||||
typedef SharedPtr<IBlockOutputStream> BlockOutputStreamPtr;
|
||||
|
||||
}
|
||||
|
@ -20,15 +20,15 @@ namespace DB
|
||||
/// Информация для профайлинга.
|
||||
struct BlockStreamProfileInfo
|
||||
{
|
||||
bool started;
|
||||
bool started = false;
|
||||
Stopwatch work_stopwatch; /// Время вычислений (выполнения функции read())
|
||||
Stopwatch total_stopwatch; /// Время с учётом ожидания
|
||||
|
||||
String stream_name; /// Короткое имя потока, для которого собирается информация
|
||||
|
||||
size_t rows;
|
||||
size_t blocks;
|
||||
size_t bytes;
|
||||
size_t rows = 0;
|
||||
size_t blocks = 0;
|
||||
size_t bytes = 0;
|
||||
|
||||
/// Информация о вложенных потоках - для выделения чистого времени работы.
|
||||
typedef std::vector<const BlockStreamProfileInfo *> BlockStreamProfileInfos;
|
||||
@ -36,12 +36,6 @@ struct BlockStreamProfileInfo
|
||||
|
||||
String column_names;
|
||||
|
||||
BlockStreamProfileInfo() :
|
||||
started(false), rows(0), blocks(0), bytes(0),
|
||||
applied_limit(false), rows_before_limit(0), calculated_rows_before_limit(false)
|
||||
{
|
||||
}
|
||||
|
||||
/// Собрать BlockStreamProfileInfo для ближайших в дереве источников с именем name. Пример; собрать все info для PartialSorting stream-ов.
|
||||
void collectInfosForStreamsWithName(const String & name, BlockStreamProfileInfos & res) const;
|
||||
|
||||
@ -64,12 +58,12 @@ private:
|
||||
void calculateRowsBeforeLimit() const;
|
||||
|
||||
/// Для этих полей сделаем accessor'ы, т.к. их необходимо предварительно вычислять.
|
||||
mutable bool applied_limit; /// Применялся ли LIMIT
|
||||
mutable size_t rows_before_limit;
|
||||
mutable bool calculated_rows_before_limit; /// Вычислялось ли поле rows_before_limit
|
||||
mutable bool applied_limit = false; /// Применялся ли LIMIT
|
||||
mutable size_t rows_before_limit = 0;
|
||||
mutable bool calculated_rows_before_limit = false; /// Вычислялось ли поле rows_before_limit
|
||||
};
|
||||
|
||||
|
||||
|
||||
/** Смотрит за тем, как работает источник блоков.
|
||||
* Позволяет получить информацию для профайлинга:
|
||||
* строк в секунду, блоков в секунду, мегабайт в секунду и т. п.
|
||||
@ -78,10 +72,6 @@ private:
|
||||
class IProfilingBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
IProfilingBlockInputStream(StoragePtr owned_storage_ = StoragePtr())
|
||||
: IBlockInputStream(owned_storage_), is_cancelled(false), process_list_elem(NULL),
|
||||
enabled_extremes(false), quota(NULL), prev_elapsed(0) {}
|
||||
|
||||
Block read();
|
||||
|
||||
/** Реализация по-умолчанию вызывает рекурсивно readSuffix() у всех детей, а затем readSuffixImpl() у себя.
|
||||
@ -205,11 +195,11 @@ public:
|
||||
|
||||
protected:
|
||||
BlockStreamProfileInfo info;
|
||||
volatile bool is_cancelled;
|
||||
volatile bool is_cancelled = false;
|
||||
ProgressCallback progress_callback;
|
||||
ProcessList::Element * process_list_elem;
|
||||
ProcessList::Element * process_list_elem = nullptr;
|
||||
|
||||
bool enabled_extremes;
|
||||
bool enabled_extremes = false;
|
||||
|
||||
/// Дополнительная информация, которая может образоваться в процессе работы.
|
||||
|
||||
@ -222,8 +212,8 @@ protected:
|
||||
|
||||
LocalLimits limits;
|
||||
|
||||
QuotaForIntervals * quota; /// Если NULL - квота не используется.
|
||||
double prev_elapsed;
|
||||
QuotaForIntervals * quota = nullptr; /// Если nullptr - квота не используется.
|
||||
double prev_elapsed = 0;
|
||||
|
||||
/// Наследники должны реализовать эту функцию.
|
||||
virtual Block readImpl() = 0;
|
||||
|
@ -19,7 +19,7 @@ class MergingSortedBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
/// limit - если не 0, то можно выдать только первые limit строк в сортированном порядке.
|
||||
MergingSortedBlockInputStream(BlockInputStreams inputs_, SortDescription & description_, size_t max_block_size_, size_t limit_ = 0)
|
||||
MergingSortedBlockInputStream(BlockInputStreams inputs_, const SortDescription & description_, size_t max_block_size_, size_t limit_ = 0)
|
||||
: description(description_), max_block_size(max_block_size_), limit(limit_), total_merged_rows(0), first(true), has_collation(false),
|
||||
num_columns(0), source_blocks(inputs_.size()), cursors(inputs_.size()), log(&Logger::get("MergingSortedBlockInputStream"))
|
||||
{
|
||||
@ -87,7 +87,7 @@ protected:
|
||||
|
||||
/// Эти методы используются в Collapsing/Summing SortedBlockInputStream-ах.
|
||||
|
||||
/// Сохранить строчку, на которую указывает cursor в row.
|
||||
/// Сохранить строчку, на которую указывает cursor, в row.
|
||||
template<class TSortCursor>
|
||||
void setRow(Row & row, TSortCursor & cursor)
|
||||
{
|
||||
|
@ -23,13 +23,19 @@ public:
|
||||
{
|
||||
if (database.empty())
|
||||
database = context.getCurrentDatabase();
|
||||
|
||||
storage = context.getTable(database, table);
|
||||
addTableLock(storage->lockStructure(true));
|
||||
|
||||
Dependencies dependencies = context.getDependencies(DatabaseAndTableName(database, table));
|
||||
for (size_t i = 0; i < dependencies.size(); ++i)
|
||||
{
|
||||
children.push_back(new PushingToViewsBlockOutputStream(dependencies[i].first, dependencies[i].second, context, ASTPtr()));
|
||||
queries.push_back(dynamic_cast<StorageView &>(*context.getTable(dependencies[i].first, dependencies[i].second)).getInnerQuery());
|
||||
}
|
||||
|
||||
if (storage->getName() != "View")
|
||||
output = storage->write(query_ptr);
|
||||
}
|
||||
|
||||
String getName() const { return "PushingToViewsBlockOutputStream"; }
|
||||
@ -44,13 +50,13 @@ public:
|
||||
copyData(*data, *children[i]);
|
||||
}
|
||||
|
||||
Block buf(block);
|
||||
if (storage->getName() != "View")
|
||||
storage->write(query_ptr)->write(buf);
|
||||
if (output)
|
||||
output->write(block);
|
||||
}
|
||||
|
||||
private:
|
||||
StoragePtr storage;
|
||||
BlockOutputStreamPtr output;
|
||||
String database;
|
||||
String table;
|
||||
Context context;
|
||||
|
@ -15,51 +15,40 @@ namespace DB
|
||||
*/
|
||||
class RemoteBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
private:
|
||||
void init(const Settings * settings_)
|
||||
{
|
||||
if (settings_)
|
||||
{
|
||||
send_settings = true;
|
||||
settings = *settings_;
|
||||
}
|
||||
else
|
||||
send_settings = false;
|
||||
}
|
||||
public:
|
||||
/// Принимает готовое соединение.
|
||||
RemoteBlockInputStream(Connection & connection_, const String & query_, const Settings * settings_,
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete)
|
||||
: connection(connection_), query(query_), stage(stage_),
|
||||
sent_query(false), finished(false), was_cancelled(false), got_exception_from_server(false),
|
||||
log(&Logger::get("RemoteBlockInputStream (" + connection.getServerAddress() + ")"))
|
||||
const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete)
|
||||
: connection(&connection_), query(query_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
if (settings_)
|
||||
{
|
||||
send_settings = true;
|
||||
settings = *settings_;
|
||||
}
|
||||
else
|
||||
send_settings = false;
|
||||
init(settings_);
|
||||
}
|
||||
|
||||
/// Захватывает владение соединением из пула.
|
||||
RemoteBlockInputStream(ConnectionPool::Entry pool_entry_, const String & query_, const Settings * settings_,
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete)
|
||||
: pool_entry(pool_entry_), connection(*pool_entry), query(query_),
|
||||
stage(stage_), sent_query(false), finished(false), was_cancelled(false),
|
||||
got_exception_from_server(false), log(&Logger::get("RemoteBlockInputStream (" + connection.getServerAddress() + ")"))
|
||||
/// Принимает готовое соединение. Захватывает владение соединением из пула.
|
||||
RemoteBlockInputStream(ConnectionPool::Entry & pool_entry_, const String & query_, const Settings * settings_,
|
||||
const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete)
|
||||
: pool_entry(pool_entry_), connection(&*pool_entry_), query(query_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
if (settings_)
|
||||
{
|
||||
send_settings = true;
|
||||
settings = *settings_;
|
||||
}
|
||||
else
|
||||
send_settings = false;
|
||||
init(settings_);
|
||||
}
|
||||
|
||||
RemoteBlockInputStream(ConnectionPool::Entry pool_entry_, const String & query_, const Settings * settings_,
|
||||
const String & _host_column_, const String & _port_column_, QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete)
|
||||
: pool_entry(pool_entry_), connection(*pool_entry), query(query_), _host_column(_host_column_),
|
||||
_port_column(_port_column_), stage(stage_), sent_query(false), finished(false), was_cancelled(false),
|
||||
got_exception_from_server(false), log(&Logger::get("RemoteBlockInputStream (" + connection.getServerAddress() + ")"))
|
||||
/// Принимает пул, из которого нужно будет достать соединение.
|
||||
RemoteBlockInputStream(IConnectionPool * pool_, const String & query_, const Settings * settings_,
|
||||
const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete)
|
||||
: pool(pool_), query(query_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
if (settings_)
|
||||
{
|
||||
send_settings = true;
|
||||
settings = *settings_;
|
||||
}
|
||||
else
|
||||
send_settings = false;
|
||||
init(settings_);
|
||||
}
|
||||
|
||||
|
||||
@ -87,10 +76,10 @@ public:
|
||||
|
||||
if (sent_query && !was_cancelled && !finished && !got_exception_from_server)
|
||||
{
|
||||
LOG_TRACE(log, "Cancelling query");
|
||||
LOG_TRACE(log, "(" + connection->getServerAddress() + ") Cancelling query");
|
||||
|
||||
/// Если запрошено прервать запрос - попросим удалённый сервер тоже прервать запрос.
|
||||
connection.sendCancel();
|
||||
connection->sendCancel();
|
||||
was_cancelled = true;
|
||||
}
|
||||
}
|
||||
@ -102,47 +91,54 @@ public:
|
||||
* чтобы оно не осталось висеть в рассихронизированном состоянии.
|
||||
*/
|
||||
if (sent_query && !finished)
|
||||
connection.disconnect();
|
||||
connection->disconnect();
|
||||
}
|
||||
|
||||
protected:
|
||||
void populateBlock(Block & res)
|
||||
/// Отправить на удаленные сервера все временные таблицы
|
||||
void sendExternalTables()
|
||||
{
|
||||
if (!_host_column.empty() && !res.has(_host_column))
|
||||
ExternalTablesData res;
|
||||
Tables::const_iterator it;
|
||||
for (it = external_tables.begin(); it != external_tables.end(); it ++)
|
||||
{
|
||||
ColumnPtr column_ptr = ColumnConst<String>(res.rows(), connection.getHost(), new DataTypeString).convertToFullColumn();
|
||||
ColumnWithNameAndType column(column_ptr, new DataTypeString, _host_column);
|
||||
res.insert(column);
|
||||
}
|
||||
if (!_port_column.empty() && !res.has(_port_column))
|
||||
{
|
||||
ColumnPtr column_ptr = ColumnConst<UInt16>(res.rows(), connection.getPort(), new DataTypeUInt16).convertToFullColumn();
|
||||
ColumnWithNameAndType column(column_ptr, new DataTypeUInt16, _port_column);
|
||||
res.insert(column);
|
||||
StoragePtr cur = it->second;
|
||||
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
|
||||
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), settings, stage, DEFAULT_BLOCK_SIZE, 1);
|
||||
if (input.size() == 0)
|
||||
res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), it->first));
|
||||
else
|
||||
res.push_back(std::make_pair(input[0], it->first));
|
||||
}
|
||||
connection->sendExternalTablesData(res);
|
||||
}
|
||||
|
||||
Block readImpl()
|
||||
{
|
||||
if (!sent_query)
|
||||
{
|
||||
connection.sendQuery(query, "", stage, send_settings ? &settings : NULL);
|
||||
/// Если надо - достаём соединение из пула.
|
||||
if (pool)
|
||||
{
|
||||
pool_entry = pool->get(send_settings ? &settings : nullptr);
|
||||
connection = &*pool_entry;
|
||||
}
|
||||
|
||||
connection->sendQuery(query, "", stage, send_settings ? &settings : nullptr, true);
|
||||
sendExternalTables();
|
||||
sent_query = true;
|
||||
}
|
||||
|
||||
while (true)
|
||||
{
|
||||
Connection::Packet packet = connection.receivePacket();
|
||||
Connection::Packet packet = connection->receivePacket();
|
||||
|
||||
switch (packet.type)
|
||||
{
|
||||
case Protocol::Server::Data:
|
||||
/// Если блок не пуст и не является заголовочным блоком
|
||||
if (packet.block && packet.block.rows() > 0)
|
||||
{
|
||||
populateBlock(packet.block);
|
||||
return packet.block;
|
||||
}
|
||||
break; /// Если блок пустой - получим другие пакеты до EndOfStream.
|
||||
|
||||
case Protocol::Server::Exception:
|
||||
@ -204,16 +200,16 @@ protected:
|
||||
/// Отправим просьбу прервать выполнение запроса, если ещё не отправляли.
|
||||
if (!was_cancelled)
|
||||
{
|
||||
LOG_TRACE(log, "Cancelling query because enough data has been read");
|
||||
LOG_TRACE(log, "(" + connection->getServerAddress() + ") Cancelling query because enough data has been read");
|
||||
|
||||
was_cancelled = true;
|
||||
connection.sendCancel();
|
||||
connection->sendCancel();
|
||||
}
|
||||
|
||||
/// Получим оставшиеся пакеты, чтобы не было рассинхронизации в соединении с сервером.
|
||||
while (true)
|
||||
{
|
||||
Connection::Packet packet = connection.receivePacket();
|
||||
Connection::Packet packet = connection->receivePacket();
|
||||
|
||||
switch (packet.type)
|
||||
{
|
||||
@ -241,40 +237,37 @@ protected:
|
||||
}
|
||||
|
||||
private:
|
||||
/// Используется, если нужно владеть соединением из пула
|
||||
IConnectionPool * pool = nullptr;
|
||||
ConnectionPool::Entry pool_entry;
|
||||
|
||||
Connection & connection;
|
||||
Connection * connection = nullptr;
|
||||
|
||||
const String query;
|
||||
bool send_settings;
|
||||
Settings settings;
|
||||
/// Имя столбца, куда записать имя хоста (Например "_host"). Пустая строка, если записывать не надо.
|
||||
String _host_column;
|
||||
/// Имя столбца, куда записать номер порта (Например "_port"). Пустая строка, если записывать не надо.
|
||||
String _port_column;
|
||||
/// Временные таблицы, которые необходимо переслать на удаленные сервера.
|
||||
Tables external_tables;
|
||||
QueryProcessingStage::Enum stage;
|
||||
|
||||
/// Отправили запрос (это делается перед получением первого блока).
|
||||
bool sent_query;
|
||||
bool sent_query = false;
|
||||
|
||||
/** Получили все данные от сервера, до пакета EndOfStream.
|
||||
* Если при уничтожении объекта, ещё не все данные считаны,
|
||||
* то для того, чтобы не было рассинхронизации, на сервер отправляется просьба прервать выполнение запроса,
|
||||
* и после этого считываются все пакеты до EndOfStream.
|
||||
*/
|
||||
bool finished;
|
||||
bool finished = false;
|
||||
|
||||
/** На сервер была отправлена просьба прервать выполенение запроса, так как данные больше не нужны.
|
||||
* Это может быть из-за того, что данных достаточно (например, при использовании LIMIT),
|
||||
* или если на стороне клиента произошло исключение.
|
||||
*/
|
||||
bool was_cancelled;
|
||||
bool was_cancelled = false;
|
||||
|
||||
/// С сервера было получено исключение. В этом случае получать больше пакетов или просить прервать запрос не нужно.
|
||||
bool got_exception_from_server;
|
||||
bool got_exception_from_server = false;
|
||||
|
||||
Logger * log;
|
||||
Logger * log = &Logger::get("RemoteBlockInputStream");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -18,7 +18,7 @@ namespace DB
|
||||
class SummingSortedBlockInputStream : public MergingSortedBlockInputStream
|
||||
{
|
||||
public:
|
||||
SummingSortedBlockInputStream(BlockInputStreams inputs_, SortDescription & description_, size_t max_block_size_)
|
||||
SummingSortedBlockInputStream(BlockInputStreams inputs_, const SortDescription & description_, size_t max_block_size_)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_),
|
||||
log(&Logger::get("SummingSortedBlockInputStream")), current_row_is_zero(false)
|
||||
{
|
||||
|
@ -38,13 +38,7 @@ public:
|
||||
return res.str();
|
||||
}
|
||||
|
||||
const Block & getTotals()
|
||||
{
|
||||
if (totals && expression)
|
||||
expression->execute(totals);
|
||||
|
||||
return totals;
|
||||
}
|
||||
const Block & getTotals();
|
||||
|
||||
protected:
|
||||
Block readImpl();
|
||||
|
@ -11,19 +11,19 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
/** Тип - состояние агрегатной функции.
|
||||
* Параметры типа - это агрегатная функция и типы её аргументов.
|
||||
* Параметры типа - это агрегатная функция, типы её аргументов и её параметры (для параметрических агрегатных функций).
|
||||
*/
|
||||
class DataTypeAggregateFunction : public IDataType
|
||||
{
|
||||
private:
|
||||
AggregateFunctionPtr function;
|
||||
DataTypes argument_types;
|
||||
Array parameters;
|
||||
|
||||
public:
|
||||
DataTypeAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_)
|
||||
: function(function_), argument_types(argument_types_)
|
||||
DataTypeAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_)
|
||||
: function(function_), argument_types(argument_types_), parameters(parameters_)
|
||||
{
|
||||
function->setArguments(argument_types);
|
||||
}
|
||||
|
||||
std::string getName() const
|
||||
@ -31,6 +31,18 @@ public:
|
||||
std::stringstream stream;
|
||||
stream << "AggregateFunction(" << function->getName();
|
||||
|
||||
if (!parameters.empty())
|
||||
{
|
||||
stream << "(";
|
||||
for (size_t i = 0; i < parameters.size(); ++i)
|
||||
{
|
||||
if (i)
|
||||
stream << ", ";
|
||||
stream << apply_visitor(DB::FieldVisitorToString(), parameters[i]);
|
||||
}
|
||||
stream << ")";
|
||||
}
|
||||
|
||||
for (DataTypes::const_iterator it = argument_types.begin(); it != argument_types.end(); ++it)
|
||||
stream << ", " << (*it)->getName();
|
||||
|
||||
@ -38,7 +50,7 @@ public:
|
||||
return stream.str();
|
||||
}
|
||||
|
||||
DataTypePtr clone() const { return new DataTypeAggregateFunction(function, argument_types); }
|
||||
DataTypePtr clone() const { return new DataTypeAggregateFunction(function, argument_types, parameters); }
|
||||
|
||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const;
|
||||
void deserializeBinary(Field & field, ReadBuffer & istr) const;
|
||||
|
@ -17,7 +17,7 @@ private:
|
||||
|
||||
public:
|
||||
/// Некоторые типы могут быть еще не известны.
|
||||
DataTypeExpression(DataTypes argument_types_ = DataTypes(), DataTypePtr return_type_ = NULL)
|
||||
DataTypeExpression(DataTypes argument_types_ = DataTypes(), DataTypePtr return_type_ = nullptr)
|
||||
: argument_types(argument_types_), return_type(return_type_) {}
|
||||
|
||||
std::string getName() const
|
||||
|
@ -10,6 +10,8 @@
|
||||
#include <DB/Interpreters/HashMap.h>
|
||||
#include <DB/Interpreters/ClearableHashMap.h>
|
||||
#include <DB/Interpreters/AggregationCommon.h>
|
||||
#include <DB/Functions/NumberTraits.h>
|
||||
#include <DB/Functions/FunctionsConditional.h>
|
||||
|
||||
#include <unordered_map>
|
||||
|
||||
@ -30,26 +32,128 @@ namespace DB
|
||||
* Например: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]
|
||||
*/
|
||||
|
||||
|
||||
|
||||
|
||||
class FunctionArray : public IFunction
|
||||
{
|
||||
public:
|
||||
private:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return "array";
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool checkRightType(DataTypePtr left, DataTypePtr right, DataTypePtr & type_res) const
|
||||
{
|
||||
if (dynamic_cast<const T1 *>(&*right))
|
||||
{
|
||||
typedef typename NumberTraits::ResultOfIf<typename T0::FieldType, typename T1::FieldType>::Type ResultType;
|
||||
type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
|
||||
if (!type_res)
|
||||
throw Exception("Arguments of function " + getName() + " are not upscalable to a common type without loss of precision.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool checkLeftType(DataTypePtr left, DataTypePtr right, DataTypePtr & type_res) const
|
||||
{
|
||||
if (dynamic_cast<const T0 *>(&*left))
|
||||
{
|
||||
if ( checkRightType<T0, DataTypeUInt8>(left, right, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt16>(left, right, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt32>(left, right, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt64>(left, right, type_res)
|
||||
|| checkRightType<T0, DataTypeInt8>(left, right, type_res)
|
||||
|| checkRightType<T0, DataTypeInt16>(left, right, type_res)
|
||||
|| checkRightType<T0, DataTypeInt32>(left, right, type_res)
|
||||
|| checkRightType<T0, DataTypeInt64>(left, right, type_res)
|
||||
|| checkRightType<T0, DataTypeFloat32>(left, right, type_res)
|
||||
|| checkRightType<T0, DataTypeFloat64>(left, right, type_res))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal type " + right->getName() + " as argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool tryAddField(DataTypePtr type_res, const Field & f, Array & arr) const
|
||||
{
|
||||
if (dynamic_cast<const T0 *>(&*type_res))
|
||||
{
|
||||
arr.push_back(apply_visitor(FieldVisitorConvertToNumber<typename T1::FieldType>(), f));
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool addField(DataTypePtr type_res, const Field & f, Array & arr) const
|
||||
{
|
||||
/// Иначе необходимо
|
||||
if ( tryAddField<DataTypeUInt8, DataTypeUInt64>(type_res, f, arr)
|
||||
|| tryAddField<DataTypeUInt16, DataTypeUInt64>(type_res, f, arr)
|
||||
|| tryAddField<DataTypeUInt32, DataTypeUInt64>(type_res, f, arr)
|
||||
|| tryAddField<DataTypeUInt64, DataTypeUInt64>(type_res, f, arr)
|
||||
|| tryAddField<DataTypeInt8, DataTypeInt64>(type_res, f, arr)
|
||||
|| tryAddField<DataTypeInt16, DataTypeInt64>(type_res, f, arr)
|
||||
|| tryAddField<DataTypeInt32, DataTypeInt64>(type_res, f, arr)
|
||||
|| tryAddField<DataTypeInt64, DataTypeInt64>(type_res, f, arr)
|
||||
|| tryAddField<DataTypeFloat32, DataTypeFloat64>(type_res, f, arr)
|
||||
|| tryAddField<DataTypeFloat64, DataTypeFloat64>(type_res, f, arr) )
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal result type " + type_res->getName() + " of function " + getName(),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
DataTypePtr getLeastCommonType(DataTypePtr left, DataTypePtr right) const
|
||||
{
|
||||
DataTypePtr type_res;
|
||||
if (!( checkLeftType<DataTypeUInt8>(left, right, type_res)
|
||||
|| checkLeftType<DataTypeUInt16>(left, right, type_res)
|
||||
|| checkLeftType<DataTypeUInt32>(left, right, type_res)
|
||||
|| checkLeftType<DataTypeUInt64>(left, right, type_res)
|
||||
|| checkLeftType<DataTypeInt8>(left, right, type_res)
|
||||
|| checkLeftType<DataTypeInt16>(left, right, type_res)
|
||||
|| checkLeftType<DataTypeInt32>(left, right, type_res)
|
||||
|| checkLeftType<DataTypeInt64>(left, right, type_res)
|
||||
|| checkLeftType<DataTypeFloat32>(left, right, type_res)
|
||||
|| checkLeftType<DataTypeFloat64>(left, right, type_res)))
|
||||
throw Exception("Internal error: unexpected type " + left->getName() + " as argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
return type_res;
|
||||
}
|
||||
|
||||
public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.empty())
|
||||
throw Exception("Function array requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (size_t i = 1, size = arguments.size(); i < size; ++i)
|
||||
if (arguments[i]->getName() != arguments[0]->getName())
|
||||
throw Exception("Arguments for function array must have same type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
DataTypePtr result_type = arguments[0];
|
||||
|
||||
return new DataTypeArray(arguments[0]);
|
||||
if (result_type->behavesAsNumber())
|
||||
{
|
||||
/// Если тип числовой, пробуем выделить наименьший общий тип
|
||||
for (size_t i = 1, size = arguments.size(); i < size; ++i)
|
||||
result_type = getLeastCommonType(result_type, arguments[i]);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Иначе все аргументы должны быть одинаковыми
|
||||
for (size_t i = 1, size = arguments.size(); i < size; ++i)
|
||||
if (arguments[i]->getName() != arguments[0]->getName())
|
||||
throw Exception("Arguments for function array must have same type or behave as number.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return new DataTypeArray(result_type);
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
@ -60,22 +164,37 @@ public:
|
||||
if (!block.getByPosition(arguments[i]).column->isConst())
|
||||
throw Exception("Arguments for function array must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
DataTypePtr result_type = block.getByPosition(arguments[0]).type;
|
||||
if (result_type->behavesAsNumber())
|
||||
{
|
||||
/// Если тип числовой, вычисляем наименьший общий тип
|
||||
for (size_t i = 1, size = arguments.size(); i < size; ++i)
|
||||
result_type = getLeastCommonType(result_type, block.getByPosition(arguments[i]).type);
|
||||
}
|
||||
|
||||
Array arr;
|
||||
for (size_t i = 0, size = arguments.size(); i < size; ++i)
|
||||
arr.push_back((*block.getByPosition(arguments[i]).column)[0]);
|
||||
if (block.getByPosition(arguments[i]).type->getName() == result_type->getName())
|
||||
/// Если элемент такого же типа как результат, просто добавляем его в ответ
|
||||
arr.push_back((*block.getByPosition(arguments[i]).column)[0]);
|
||||
else
|
||||
/// Иначе необходимо привести его к типу результата
|
||||
addField(result_type, (*block.getByPosition(arguments[i]).column)[0], arr);
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstArray(block.getByPosition(arguments[0]).column->size(), arr, new DataTypeArray(block.getByPosition(arguments[0]).type));
|
||||
block.getByPosition(result).column = new ColumnConstArray(block.getByPosition(arguments[0]).column->size(), arr, new DataTypeArray(result_type));
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename T, bool negative>
|
||||
template <typename T>
|
||||
struct ArrayElementNumImpl
|
||||
{
|
||||
/** Если negative = false - передаётся индекс с начала массива, начиная с нуля.
|
||||
/** Процедура для константного идекса
|
||||
* Если negative = false - передаётся индекс с начала массива, начиная с нуля.
|
||||
* Если negative = true - передаётся индекс с конца массива, начиная с нуля.
|
||||
*/
|
||||
static void vector(
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
const PODArray<T> & data, const ColumnArray::Offsets_t & offsets,
|
||||
const ColumnArray::Offset_t index,
|
||||
PODArray<T> & result)
|
||||
@ -96,12 +215,58 @@ struct ArrayElementNumImpl
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
/** Процедура для неконстантного идекса
|
||||
* index_type - тип данных идекса
|
||||
*/
|
||||
template <typename index_type>
|
||||
static void vector(
|
||||
const PODArray<T> & data, const ColumnArray::Offsets_t & offsets,
|
||||
const ColumnVector<index_type> & index,
|
||||
PODArray<T> & result)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result.resize(size);
|
||||
|
||||
ColumnArray::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
|
||||
if (index[i].getType() == Field::Types::UInt64)
|
||||
{
|
||||
UInt64 cur_id = safeGet<UInt64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
result[i] = data[current_offset + cur_id - 1];
|
||||
else
|
||||
result[i] = T();
|
||||
}
|
||||
else if (index[i].getType() == Field::Types::Int64)
|
||||
{
|
||||
Int64 cur_id = safeGet<Int64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
result[i] = data[current_offset + cur_id - 1];
|
||||
else if (cur_id < 0 && -cur_id <= array_size)
|
||||
result[i] = data[offsets[i] + cur_id];
|
||||
else
|
||||
result[i] = T();
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <bool negative>
|
||||
struct ArrayElementStringImpl
|
||||
{
|
||||
static void vector(
|
||||
/** Процедура для константного идекса
|
||||
* Если negative = false - передаётся индекс с начала массива, начиная с нуля.
|
||||
* Если negative = true - передаётся индекс с конца массива, начиная с нуля.
|
||||
*/
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
|
||||
const ColumnArray::Offset_t index,
|
||||
ColumnString::Chars_t & result_data, ColumnArray::Offsets_t & result_offsets)
|
||||
@ -143,6 +308,73 @@ struct ArrayElementStringImpl
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
/** Процедура для неконстантного идекса
|
||||
* index_type - тип данных идекса
|
||||
*/
|
||||
template <typename index_type>
|
||||
static void vector(
|
||||
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
|
||||
const ColumnVector<index_type> & index,
|
||||
ColumnString::Chars_t & result_data, ColumnArray::Offsets_t & result_offsets)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result_offsets.resize(size);
|
||||
result_data.reserve(data.size());
|
||||
|
||||
ColumnArray::Offset_t current_offset = 0;
|
||||
ColumnArray::Offset_t current_result_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
size_t adjusted_index;
|
||||
|
||||
if (index[i].getType() == Field::Types::UInt64)
|
||||
{
|
||||
UInt64 cur_id = safeGet<UInt64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
adjusted_index = cur_id - 1;
|
||||
else
|
||||
adjusted_index = array_size; /// Индекс не вписывается в рамки массива, заменяем заведомо слишком большим
|
||||
}
|
||||
else if (index[i].getType() == Field::Types::Int64)
|
||||
{
|
||||
Int64 cur_id = safeGet<Int64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
adjusted_index = cur_id - 1;
|
||||
else if (cur_id < 0 && -cur_id <= array_size)
|
||||
adjusted_index = array_size + cur_id;
|
||||
else
|
||||
adjusted_index = array_size; /// Индекс не вписывается в рамки массива, заменяем слишком большим
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (adjusted_index < array_size)
|
||||
{
|
||||
ColumnArray::Offset_t string_pos = current_offset == 0 && adjusted_index == 0
|
||||
? 0
|
||||
: string_offsets[current_offset + adjusted_index - 1];
|
||||
|
||||
ColumnArray::Offset_t string_size = string_offsets[current_offset + adjusted_index] - string_pos;
|
||||
|
||||
result_data.resize(current_result_offset + string_size);
|
||||
memcpy(&result_data[current_result_offset], &data[string_pos], string_size);
|
||||
current_result_offset += string_size;
|
||||
result_offsets[i] = current_result_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Вставим пустую строку.
|
||||
result_data.resize(current_result_offset + 1);
|
||||
result_data[current_result_offset] = 0;
|
||||
current_result_offset += 1;
|
||||
result_offsets[i] = current_result_offset;
|
||||
}
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -150,7 +382,7 @@ class FunctionArrayElement : public IFunction
|
||||
{
|
||||
private:
|
||||
template <typename T>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnArray * col_array = dynamic_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
@ -166,16 +398,37 @@ private:
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
if (index.getType() == Field::Types::UInt64)
|
||||
ArrayElementNumImpl<T, false>::vector(col_nested->getData(), col_array->getOffsets(), safeGet<UInt64>(index) - 1, col_res->getData());
|
||||
ArrayElementNumImpl<T>::template vectorConst<false>(col_nested->getData(), col_array->getOffsets(), safeGet<UInt64>(index) - 1, col_res->getData());
|
||||
else if (index.getType() == Field::Types::Int64)
|
||||
ArrayElementNumImpl<T, true>::vector(col_nested->getData(), col_array->getOffsets(), -safeGet<Int64>(index) - 1, col_res->getData());
|
||||
ArrayElementNumImpl<T>::template vectorConst<true>(col_nested->getData(), col_array->getOffsets(), -safeGet<Int64>(index) - 1, col_res->getData());
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
template <typename index_type, typename data_type>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<index_type> & index)
|
||||
{
|
||||
const ColumnArray * col_array = dynamic_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnVector<data_type> * col_nested = dynamic_cast<const ColumnVector<data_type> *>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
ColumnVector<data_type> * col_res = new ColumnVector<data_type>;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ArrayElementNumImpl<data_type>::template vector<index_type>(col_nested->getData(), col_array->getOffsets(), index, col_res->getData());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnArray * col_array = dynamic_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
@ -191,7 +444,7 @@ private:
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
if (index.getType() == Field::Types::UInt64)
|
||||
ArrayElementStringImpl<false>::vector(
|
||||
ArrayElementStringImpl::vectorConst<false>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
@ -199,7 +452,7 @@ private:
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets());
|
||||
else if (index.getType() == Field::Types::Int64)
|
||||
ArrayElementStringImpl<true>::vector(
|
||||
ArrayElementStringImpl::vectorConst<true>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
@ -212,7 +465,34 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
template <typename index_type>
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<index_type> & index)
|
||||
{
|
||||
const ColumnArray * col_array = dynamic_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnString * col_nested = dynamic_cast<const ColumnString *>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
ColumnString * col_res = new ColumnString;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ArrayElementStringImpl::vector<index_type>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
index,
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeConstConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnConstArray * col_array = dynamic_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
@ -239,6 +519,71 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename index_type>
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<index_type> & index)
|
||||
{
|
||||
const ColumnConstArray * col_array = dynamic_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const DB::Array & array = col_array->getData();
|
||||
size_t array_size = array.size();
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createColumn();
|
||||
|
||||
for (size_t i = 0; i < col_array->size(); ++i)
|
||||
{
|
||||
if (index[i].getType() == Field::Types::UInt64)
|
||||
{
|
||||
UInt64 cur_id = safeGet<UInt64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
block.getByPosition(result).column->insert(array[cur_id - 1]);
|
||||
else
|
||||
block.getByPosition(result).column->insertDefault();
|
||||
}
|
||||
else if (index[i].getType() == Field::Types::Int64)
|
||||
{
|
||||
Int64 cur_id = safeGet<Int64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
block.getByPosition(result).column->insert(array[cur_id - 1]);
|
||||
else if (cur_id < 0 && -cur_id <= array_size)
|
||||
block.getByPosition(result).column->insert(array[array_size + cur_id]);
|
||||
else
|
||||
block.getByPosition(result).column->insertDefault();
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename index_type>
|
||||
bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnVector<index_type> * index = dynamic_cast<const ColumnVector<index_type> *> (&*block.getByPosition(arguments[1]).column);
|
||||
|
||||
if (!index)
|
||||
return false;
|
||||
|
||||
if (!( executeNumber<index_type, UInt8> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, UInt16> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, UInt32> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, UInt64> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Int8> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Int16> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Int32> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Int64> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Float32> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Float64> (block, arguments, result, *index)
|
||||
|| executeConst <index_type> (block, arguments, result, *index)
|
||||
|| executeString<index_type> (block, arguments, result, *index)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return true;
|
||||
}
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
@ -269,28 +614,41 @@ public:
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (!block.getByPosition(arguments[1]).column->isConst())
|
||||
throw Exception("Second argument for function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
{
|
||||
if (!( executeArgument<UInt8> (block, arguments, result)
|
||||
|| executeArgument<UInt16> (block, arguments, result)
|
||||
|| executeArgument<UInt32> (block, arguments, result)
|
||||
|| executeArgument<UInt64> (block, arguments, result)
|
||||
|| executeArgument<Int8> (block, arguments, result)
|
||||
|| executeArgument<Int16> (block, arguments, result)
|
||||
|| executeArgument<Int32> (block, arguments, result)
|
||||
|| executeArgument<Int64> (block, arguments, result)))
|
||||
throw Exception("Second argument for function " + getName() + " must must have UInt or Int type.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
{
|
||||
Field index = (*block.getByPosition(arguments[1]).column)[0];
|
||||
|
||||
Field index = (*block.getByPosition(arguments[1]).column)[0];
|
||||
if (index == UInt64(0))
|
||||
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
|
||||
if (index == UInt64(0))
|
||||
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
|
||||
if (!( executeNumber<UInt8> (block, arguments, result, index)
|
||||
|| executeNumber<UInt16> (block, arguments, result, index)
|
||||
|| executeNumber<UInt32> (block, arguments, result, index)
|
||||
|| executeNumber<UInt64> (block, arguments, result, index)
|
||||
|| executeNumber<Int8> (block, arguments, result, index)
|
||||
|| executeNumber<Int16> (block, arguments, result, index)
|
||||
|| executeNumber<Int32> (block, arguments, result, index)
|
||||
|| executeNumber<Int64> (block, arguments, result, index)
|
||||
|| executeNumber<Float32> (block, arguments, result, index)
|
||||
|| executeNumber<Float64> (block, arguments, result, index)
|
||||
|| executeConst (block, arguments, result, index)
|
||||
|| executeString (block, arguments, result, index)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (!( executeNumberConst<UInt8> (block, arguments, result, index)
|
||||
|| executeNumberConst<UInt16> (block, arguments, result, index)
|
||||
|| executeNumberConst<UInt32> (block, arguments, result, index)
|
||||
|| executeNumberConst<UInt64> (block, arguments, result, index)
|
||||
|| executeNumberConst<Int8> (block, arguments, result, index)
|
||||
|| executeNumberConst<Int16> (block, arguments, result, index)
|
||||
|| executeNumberConst<Int32> (block, arguments, result, index)
|
||||
|| executeNumberConst<Int64> (block, arguments, result, index)
|
||||
|| executeNumberConst<Float32> (block, arguments, result, index)
|
||||
|| executeNumberConst<Float64> (block, arguments, result, index)
|
||||
|| executeConstConst (block, arguments, result, index)
|
||||
|| executeStringConst (block, arguments, result, index)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
@ -633,7 +991,7 @@ public:
|
||||
return;
|
||||
|
||||
Columns array_columns(arguments.size());
|
||||
const ColumnArray::Offsets_t * offsets = NULL;
|
||||
const ColumnArray::Offsets_t * offsets = nullptr;
|
||||
ConstColumnPlainPtrs data_columns(arguments.size());
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
|
@ -289,7 +289,7 @@ struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
|
||||
{
|
||||
static DataTypePtr getDataType()
|
||||
{
|
||||
return NULL;
|
||||
return nullptr;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -17,9 +17,7 @@ namespace DB
|
||||
{
|
||||
|
||||
/** Функции преобразования типов.
|
||||
*
|
||||
* Бывают двух видов:
|
||||
* - toType - преобразование "естественным образом"; TODO: преобразования из/в FixedString.
|
||||
* toType - преобразование "естественным образом";
|
||||
*/
|
||||
|
||||
|
||||
@ -229,7 +227,7 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
parseImpl<ToDataType>(vec_to[i], read_buffer);
|
||||
readChar(zero, read_buffer);
|
||||
if (zero != 0)
|
||||
throw Exception("Cannot parse number from string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||
throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||
}
|
||||
}
|
||||
else if (const ColumnConstString * col_from = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column))
|
||||
@ -247,7 +245,6 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Если типы совпадают - просто скопируем ссылку на столбец.
|
||||
*/
|
||||
template <typename Name>
|
||||
@ -260,6 +257,110 @@ struct ConvertImpl<DataTypeString, DataTypeString, Name>
|
||||
};
|
||||
|
||||
|
||||
/** Преобразование из FixedString.
|
||||
*/
|
||||
template <typename ToDataType, typename Name>
|
||||
struct ConvertImpl<DataTypeFixedString, ToDataType, Name>
|
||||
{
|
||||
typedef typename ToDataType::FieldType ToFieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (const ColumnFixedString * col_from = dynamic_cast<const ColumnFixedString *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnVector<ToFieldType> * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const ColumnFixedString::Chars_t & data_from = col_from->getChars();
|
||||
size_t n = col_from->getN();
|
||||
typename ColumnVector<ToFieldType>::Container_t & vec_to = col_to->getData();
|
||||
size_t size = col_from->size();
|
||||
vec_to.resize(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
char * begin = const_cast<char *>(reinterpret_cast<const char *>(&data_from[i * n]));
|
||||
char * end = begin + n;
|
||||
ReadBuffer read_buffer(begin, n, 0);
|
||||
parseImpl<ToDataType>(vec_to[i], read_buffer);
|
||||
|
||||
if (!read_buffer.eof())
|
||||
{
|
||||
while (read_buffer.position() < end && *read_buffer.position() == 0)
|
||||
++read_buffer.position();
|
||||
|
||||
if (read_buffer.position() < end)
|
||||
throw Exception("Cannot parse from fixed string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (const ColumnConstString * col_from = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::get(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
/** Преобразование из FixedString в String.
|
||||
* При этом, вырезаются последовательности нулевых байт с конца строк.
|
||||
*/
|
||||
template <typename Name>
|
||||
struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
{
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (const ColumnFixedString * col_from = dynamic_cast<const ColumnFixedString *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnString * col_to = new ColumnString;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const ColumnFixedString::Chars_t & data_from = col_from->getChars();
|
||||
ColumnString::Chars_t & data_to = col_to->getChars();
|
||||
ColumnString::Offsets_t & offsets_to = col_to->getOffsets();
|
||||
size_t size = col_from->size();
|
||||
size_t n = col_from->getN();
|
||||
data_to.resize(size * (n + 1)); /// + 1 - нулевой байт
|
||||
offsets_to.resize(size);
|
||||
|
||||
size_t offset_from = 0;
|
||||
size_t offset_to = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t bytes_to_copy = n;
|
||||
while (bytes_to_copy > 0 && data_from[offset_from + bytes_to_copy - 1] == 0)
|
||||
--bytes_to_copy;
|
||||
|
||||
memcpy(&data_to[offset_to], &data_from[offset_from], bytes_to_copy);
|
||||
offset_from += n;
|
||||
offset_to += bytes_to_copy;
|
||||
data_to[offset_to] = 0;
|
||||
++offset_to;
|
||||
offsets_to[i] = offset_to;
|
||||
}
|
||||
|
||||
data_to.resize(offset_to);
|
||||
}
|
||||
else if (const ColumnConstString * col_from = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
const String & s = col_from->getData();
|
||||
|
||||
size_t bytes_to_copy = s.size();
|
||||
while (bytes_to_copy > 0 && s[bytes_to_copy - 1] == 0)
|
||||
--bytes_to_copy;
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstString(col_from->size(), s.substr(0, bytes_to_copy));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::get(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename ToDataType, typename Name>
|
||||
class FunctionConvert : public IFunction
|
||||
@ -300,6 +401,7 @@ public:
|
||||
else if (dynamic_cast<const DataTypeDate * >(from_type)) ConvertImpl<DataTypeDate, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeDateTime * >(from_type)) ConvertImpl<DataTypeDateTime, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeString * >(from_type)) ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeFixedString *>(from_type)) ConvertImpl<DataTypeFixedString, ToDataType, Name>::execute(block, arguments, result);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -320,7 +422,7 @@ public:
|
||||
|
||||
/** Получить тип результата по типам аргументов и значениям константных аргументов.
|
||||
* Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
* Для неконстантных столбцов arguments[i].column=NULL.
|
||||
* Для неконстантных столбцов arguments[i].column = nullptr.
|
||||
*/
|
||||
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments,
|
||||
DataTypePtr & out_return_type,
|
||||
|
@ -119,15 +119,48 @@ struct CategoryHierarchyImpl
|
||||
};
|
||||
|
||||
|
||||
/** Вспомогательная вещь, позволяющая достать из словаря конкретный словарь, соответствующий точке зрения
|
||||
* (ключу словаря, передаваемому в аргументе функции).
|
||||
* Пример: при вызове regionToCountry(x, 'ua'), может быть использован словарь, в котором Крым относится к Украине.
|
||||
*/
|
||||
struct RegionsHierarchyGetter
|
||||
{
|
||||
typedef RegionsHierarchies Src;
|
||||
typedef RegionsHierarchy Dst;
|
||||
|
||||
static const Dst & get(const Src & src, const std::string & key)
|
||||
{
|
||||
return src.get(key);
|
||||
}
|
||||
};
|
||||
|
||||
/** Для словарей без поддержки ключей. Ничего не делает.
|
||||
*/
|
||||
template <typename Dict>
|
||||
struct IdentityDictionaryGetter
|
||||
{
|
||||
typedef Dict Src;
|
||||
typedef Dict Dst;
|
||||
|
||||
static const Dst & get(const Src & src, const std::string & key)
|
||||
{
|
||||
if (key.empty())
|
||||
return src;
|
||||
else
|
||||
throw Exception("Dictionary doesn't support 'point of view' keys.", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Преобразует идентификатор, используя словарь.
|
||||
template <typename T, typename Transform, typename Dict, typename Name>
|
||||
template <typename T, typename Transform, typename DictGetter, typename Name>
|
||||
class FunctionTransformWithDictionary : public IFunction
|
||||
{
|
||||
private:
|
||||
const SharedPtr<Dict> owned_dict;
|
||||
const SharedPtr<typename DictGetter::Src> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionTransformWithDictionary(const SharedPtr<Dict> & owned_dict_)
|
||||
FunctionTransformWithDictionary(const SharedPtr<typename DictGetter::Src> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
@ -143,9 +176,9 @@ public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
if (arguments.size() != 1 && arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
+ toString(arguments.size()) + ", should be 1 or 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (arguments[0]->getName() != TypeName<T>::get())
|
||||
@ -153,12 +186,35 @@ public:
|
||||
+ " (must be " + TypeName<T>::get() + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 2 && arguments[1]->getName() != TypeName<String>::get())
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of the second ('point of view') argument of function " + getName()
|
||||
+ " (must be " + TypeName<String>::get() + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
/// Ключ словаря, определяющий "точку зрения".
|
||||
std::string dict_key;
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnConstString * key_col = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments[1]).column);
|
||||
|
||||
if (!key_col)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second ('point of view') argument of function " + Name::get()
|
||||
+ ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
dict_key = key_col->getData();
|
||||
}
|
||||
|
||||
const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key);
|
||||
|
||||
if (const ColumnVector<T> * col_from = dynamic_cast<const ColumnVector<T> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnVector<T> * col_to = new ColumnVector<T>;
|
||||
@ -169,13 +225,12 @@ public:
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
const Dict & dict = *owned_dict;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::apply(vec_from[i], dict);
|
||||
}
|
||||
else if (const ColumnConst<T> * col_from = dynamic_cast<const ColumnConst<T> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<T>(col_from->size(), Transform::apply(col_from->getData(), *owned_dict));
|
||||
block.getByPosition(result).column = new ColumnConst<T>(col_from->size(), Transform::apply(col_from->getData(), dict));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -186,14 +241,14 @@ public:
|
||||
|
||||
|
||||
/// Проверяет принадлежность, используя словарь.
|
||||
template <typename T, typename Transform, typename Dict, typename Name>
|
||||
template <typename T, typename Transform, typename DictGetter, typename Name>
|
||||
class FunctionIsInWithDictionary : public IFunction
|
||||
{
|
||||
private:
|
||||
const SharedPtr<Dict> owned_dict;
|
||||
const SharedPtr<typename DictGetter::Src> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionIsInWithDictionary(const SharedPtr<Dict> & owned_dict_)
|
||||
FunctionIsInWithDictionary(const SharedPtr<typename DictGetter::Src> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
@ -209,9 +264,9 @@ public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
if (arguments.size() != 2 && arguments.size() != 3)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
+ toString(arguments.size()) + ", should be 2 or 3.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (arguments[0]->getName() != TypeName<T>::get())
|
||||
@ -224,12 +279,35 @@ public:
|
||||
+ " (must be " + TypeName<T>::get() + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 3 && arguments[2]->getName() != TypeName<String>::get())
|
||||
throw Exception("Illegal type " + arguments[2]->getName() + " of the third ('point of view') argument of function " + getName()
|
||||
+ " (must be " + TypeName<String>::get() + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeUInt8;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
/// Ключ словаря, определяющий "точку зрения".
|
||||
std::string dict_key;
|
||||
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
const ColumnConstString * key_col = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments[2]).column);
|
||||
|
||||
if (!key_col)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||
+ " of third ('point of view') argument of function " + Name::get()
|
||||
+ ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
dict_key = key_col->getData();
|
||||
}
|
||||
|
||||
const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key);
|
||||
|
||||
const ColumnVector<T> * col_vec1 = dynamic_cast<const ColumnVector<T> *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnVector<T> * col_vec2 = dynamic_cast<const ColumnVector<T> *>(&*block.getByPosition(arguments[1]).column);
|
||||
const ColumnConst<T> * col_const1 = dynamic_cast<const ColumnConst<T> *>(&*block.getByPosition(arguments[0]).column);
|
||||
@ -246,7 +324,6 @@ public:
|
||||
size_t size = vec_from1.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
const Dict & dict = *owned_dict;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::apply(vec_from1[i], vec_from2[i], dict);
|
||||
}
|
||||
@ -261,7 +338,6 @@ public:
|
||||
size_t size = vec_from1.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
const Dict & dict = *owned_dict;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::apply(vec_from1[i], const_from2, dict);
|
||||
}
|
||||
@ -276,14 +352,13 @@ public:
|
||||
size_t size = vec_from2.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
const Dict & dict = *owned_dict;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::apply(const_from1, vec_from2[i], dict);
|
||||
}
|
||||
else if (col_const1 && col_const2)
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<UInt8>(col_const1->size(),
|
||||
Transform::apply(col_const1->getData(), col_const2->getData(), *owned_dict));
|
||||
Transform::apply(col_const1->getData(), col_const2->getData(), dict));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -295,14 +370,14 @@ public:
|
||||
|
||||
|
||||
/// Получает массив идентификаторов, состоящий из исходного и цепочки родителей.
|
||||
template <typename T, typename Transform, typename Dict, typename Name>
|
||||
template <typename T, typename Transform, typename DictGetter, typename Name>
|
||||
class FunctionHierarchyWithDictionary : public IFunction
|
||||
{
|
||||
private:
|
||||
const SharedPtr<Dict> owned_dict;
|
||||
const SharedPtr<typename DictGetter::Src> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionHierarchyWithDictionary(const SharedPtr<Dict> & owned_dict_)
|
||||
FunctionHierarchyWithDictionary(const SharedPtr<typename DictGetter::Src> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
@ -318,15 +393,20 @@ public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
if (arguments.size() != 1 && arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
+ toString(arguments.size()) + ", should be 1 or 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (arguments[0]->getName() != TypeName<T>::get())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName()
|
||||
+ " (must be " + TypeName<T>::get() + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 2 && arguments[1]->getName() != TypeName<String>::get())
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of the second ('point of view') argument of function " + getName()
|
||||
+ " (must be " + TypeName<String>::get() + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeArray(arguments[0]);
|
||||
}
|
||||
@ -334,6 +414,24 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
/// Ключ словаря, определяющий "точку зрения".
|
||||
std::string dict_key;
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnConstString * key_col = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments[1]).column);
|
||||
|
||||
if (!key_col)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second ('point of view') argument of function " + Name::get()
|
||||
+ ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
dict_key = key_col->getData();
|
||||
}
|
||||
|
||||
const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key);
|
||||
|
||||
if (const ColumnVector<T> * col_from = dynamic_cast<const ColumnVector<T> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnVector<T> * col_values = new ColumnVector<T>;
|
||||
@ -348,7 +446,6 @@ public:
|
||||
res_offsets.resize(size);
|
||||
res_values.reserve(size * 4);
|
||||
|
||||
const Dict & dict = *owned_dict;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
T cur = vec_from[i];
|
||||
@ -364,7 +461,6 @@ public:
|
||||
{
|
||||
Array res;
|
||||
|
||||
const Dict & dict = *owned_dict;
|
||||
T cur = col_from->getData();
|
||||
while (cur)
|
||||
{
|
||||
@ -402,24 +498,53 @@ struct NameSEHierarchy { static const char * get() { return "SEHierarchy"; } };
|
||||
struct NameCategoryHierarchy{ static const char * get() { return "categoryHierarchy"; } };
|
||||
|
||||
|
||||
typedef FunctionTransformWithDictionary<UInt32, RegionToCityImpl, RegionsHierarchy, NameRegionToCity> FunctionRegionToCity;
|
||||
typedef FunctionTransformWithDictionary<UInt32, RegionToAreaImpl, RegionsHierarchy, NameRegionToArea> FunctionRegionToArea;
|
||||
typedef FunctionTransformWithDictionary<UInt32, RegionToCountryImpl,RegionsHierarchy, NameRegionToCountry> FunctionRegionToCountry;
|
||||
typedef FunctionTransformWithDictionary<UInt32, RegionToContinentImpl, RegionsHierarchy, NameRegionToContinent> FunctionRegionToContinent;
|
||||
typedef FunctionTransformWithDictionary<UInt8, OSToRootImpl, TechDataHierarchy, NameOSToRoot> FunctionOSToRoot;
|
||||
typedef FunctionTransformWithDictionary<UInt8, SEToRootImpl, TechDataHierarchy, NameSEToRoot> FunctionSEToRoot;
|
||||
typedef FunctionTransformWithDictionary<UInt16, CategoryToRootImpl, CategoriesHierarchy,NameCategoryToRoot> FunctionCategoryToRoot;
|
||||
typedef FunctionTransformWithDictionary<UInt16,CategoryToSecondLevelImpl,CategoriesHierarchy,NameCategoryToSecondLevel> FunctionCategoryToSecondLevel;
|
||||
typedef FunctionTransformWithDictionary
|
||||
<UInt32, RegionToCityImpl, RegionsHierarchyGetter, NameRegionToCity> FunctionRegionToCity;
|
||||
|
||||
typedef FunctionIsInWithDictionary<UInt32, RegionInImpl, RegionsHierarchy, NameRegionIn> FunctionRegionIn;
|
||||
typedef FunctionIsInWithDictionary<UInt8, OSInImpl, TechDataHierarchy, NameOSIn> FunctionOSIn;
|
||||
typedef FunctionIsInWithDictionary<UInt8, SEInImpl, TechDataHierarchy, NameSEIn> FunctionSEIn;
|
||||
typedef FunctionIsInWithDictionary<UInt16, CategoryInImpl, CategoriesHierarchy,NameCategoryIn> FunctionCategoryIn;
|
||||
typedef FunctionTransformWithDictionary
|
||||
<UInt32, RegionToAreaImpl, RegionsHierarchyGetter, NameRegionToArea> FunctionRegionToArea;
|
||||
|
||||
typedef FunctionHierarchyWithDictionary<UInt32, RegionHierarchyImpl, RegionsHierarchy, NameRegionHierarchy> FunctionRegionHierarchy;
|
||||
typedef FunctionHierarchyWithDictionary<UInt8, OSHierarchyImpl, TechDataHierarchy, NameOSHierarchy> FunctionOSHierarchy;
|
||||
typedef FunctionHierarchyWithDictionary<UInt8, SEHierarchyImpl, TechDataHierarchy, NameSEHierarchy> FunctionSEHierarchy;
|
||||
typedef FunctionHierarchyWithDictionary<UInt16, CategoryHierarchyImpl, CategoriesHierarchy, NameCategoryHierarchy>FunctionCategoryHierarchy;
|
||||
typedef FunctionTransformWithDictionary
|
||||
<UInt32, RegionToCountryImpl, RegionsHierarchyGetter, NameRegionToCountry> FunctionRegionToCountry;
|
||||
|
||||
typedef FunctionTransformWithDictionary
|
||||
<UInt32, RegionToContinentImpl, RegionsHierarchyGetter, NameRegionToContinent> FunctionRegionToContinent;
|
||||
|
||||
typedef FunctionTransformWithDictionary
|
||||
<UInt8, OSToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSToRoot> FunctionOSToRoot;
|
||||
|
||||
typedef FunctionTransformWithDictionary
|
||||
<UInt8, SEToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEToRoot> FunctionSEToRoot;
|
||||
|
||||
typedef FunctionTransformWithDictionary
|
||||
<UInt16, CategoryToRootImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryToRoot> FunctionCategoryToRoot;
|
||||
|
||||
typedef FunctionTransformWithDictionary
|
||||
<UInt16, CategoryToSecondLevelImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryToSecondLevel> FunctionCategoryToSecondLevel;
|
||||
|
||||
typedef FunctionIsInWithDictionary
|
||||
<UInt32, RegionInImpl, RegionsHierarchyGetter, NameRegionIn> FunctionRegionIn;
|
||||
|
||||
typedef FunctionIsInWithDictionary
|
||||
<UInt8, OSInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSIn> FunctionOSIn;
|
||||
|
||||
typedef FunctionIsInWithDictionary
|
||||
<UInt8, SEInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEIn> FunctionSEIn;
|
||||
|
||||
typedef FunctionIsInWithDictionary
|
||||
<UInt16, CategoryInImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryIn> FunctionCategoryIn;
|
||||
|
||||
typedef FunctionHierarchyWithDictionary
|
||||
<UInt32, RegionHierarchyImpl, RegionsHierarchyGetter, NameRegionHierarchy> FunctionRegionHierarchy;
|
||||
|
||||
typedef FunctionHierarchyWithDictionary
|
||||
<UInt8, OSHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSHierarchy> FunctionOSHierarchy;
|
||||
|
||||
typedef FunctionHierarchyWithDictionary
|
||||
<UInt8, SEHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEHierarchy> FunctionSEHierarchy;
|
||||
|
||||
typedef FunctionHierarchyWithDictionary
|
||||
<UInt16, CategoryHierarchyImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryHierarchy> FunctionCategoryHierarchy;
|
||||
|
||||
|
||||
/// Преобразует числовой идентификатор региона в имя на заданном языке, используя словарь.
|
||||
@ -478,7 +603,9 @@ public:
|
||||
+ " of the second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
|
||||
const RegionsNames & dict = *owned_dict;
|
||||
|
||||
if (const ColumnVector<UInt32> * col_from = dynamic_cast<const ColumnVector<UInt32> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnString * col_to = new ColumnString;
|
||||
@ -486,7 +613,6 @@ public:
|
||||
|
||||
const ColumnVector<UInt32>::Container_t & region_ids = col_from->getData();
|
||||
|
||||
const RegionsNames & dict = *owned_dict;
|
||||
for (size_t i = 0; i < region_ids.size(); ++i)
|
||||
{
|
||||
const DB::StringRef & name_ref = dict.getRegionName(region_ids[i], language);
|
||||
@ -496,7 +622,7 @@ public:
|
||||
else if (const ColumnConst<UInt32> * col_from = dynamic_cast<const ColumnConst<UInt32> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
UInt32 region_id = col_from->getData();
|
||||
const DB::StringRef & name_ref = owned_dict->getRegionName(region_id, language);
|
||||
const DB::StringRef & name_ref = dict.getRegionName(region_id, language);
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstString(col_from->size(), name_ref.toString());
|
||||
}
|
||||
|
@ -430,7 +430,7 @@ public:
|
||||
NameSet argument_names;
|
||||
|
||||
ColumnPtr column_first_array_ptr;
|
||||
const ColumnArray * column_first_array = NULL;
|
||||
const ColumnArray * column_first_array = nullptr;
|
||||
|
||||
/// Положим в блок аргументы выражения.
|
||||
|
||||
|
@ -91,7 +91,8 @@ struct AssociativeOperationImpl
|
||||
/// Выбрасывает N последних столбцов из in (если их меньше, то все) и кладет в result их комбинацию.
|
||||
static void execute(UInt8ColumnPtrs & in, UInt8Container & result)
|
||||
{
|
||||
if (N > in.size()){
|
||||
if (N > in.size())
|
||||
{
|
||||
AssociativeOperationImpl<Op, N - 1>::execute(in, result);
|
||||
return;
|
||||
}
|
||||
|
@ -285,14 +285,18 @@ class FunctionIn : public IFunction
|
||||
{
|
||||
private:
|
||||
bool negative;
|
||||
bool global;
|
||||
|
||||
public:
|
||||
FunctionIn(bool negative_ = false) : negative(negative_) {}
|
||||
FunctionIn(bool negative_ = false, bool global_ = false) : negative(negative_), global(global_) {}
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return negative ? "notIn" : "in";
|
||||
if (global)
|
||||
return negative ? "globalNotIn" : "globalIn";
|
||||
else
|
||||
return negative ? "notIn" : "in";
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
|
@ -238,14 +238,13 @@ public:
|
||||
class ExtractAllImpl
|
||||
{
|
||||
private:
|
||||
const OptimizedRegularExpression *re;
|
||||
const OptimizedRegularExpression * re = nullptr;
|
||||
OptimizedRegularExpression::MatchVec matches;
|
||||
size_t capture;
|
||||
|
||||
Pos pos;
|
||||
Pos end;
|
||||
public:
|
||||
ExtractAllImpl():re(NULL) { }
|
||||
/// Получить имя функции.
|
||||
static String getName() { return "extractAll"; }
|
||||
|
||||
@ -346,8 +345,8 @@ public:
|
||||
res_strings_offsets.reserve(src_offsets.size() * 5); /// Константа 5 - наугад.
|
||||
res_strings_chars.reserve(src_chars.size());
|
||||
|
||||
Pos token_begin = NULL;
|
||||
Pos token_end = NULL;
|
||||
Pos token_begin = nullptr;
|
||||
Pos token_end = nullptr;
|
||||
|
||||
size_t size = src_offsets.size();
|
||||
ColumnString::Offset_t current_src_offset = 0;
|
||||
@ -387,8 +386,8 @@ public:
|
||||
Array dst;
|
||||
|
||||
generator.set(src.data(), src.data() + src.size());
|
||||
Pos token_begin = NULL;
|
||||
Pos token_end = NULL;
|
||||
Pos token_begin = nullptr;
|
||||
Pos token_end = nullptr;
|
||||
|
||||
while (generator.get(token_begin, token_end))
|
||||
dst.push_back(String(token_begin, token_end - token_begin));
|
||||
|
@ -810,7 +810,7 @@ struct ReplaceStringImpl
|
||||
match = false;
|
||||
if (match)
|
||||
{
|
||||
replace_cnt ++;
|
||||
++replace_cnt;
|
||||
res_data += replacement;
|
||||
i = i + needle.size() - 1;
|
||||
} else
|
||||
|
@ -167,7 +167,7 @@ struct ExtractPath
|
||||
Pos pos = data;
|
||||
Pos end = pos + size;
|
||||
|
||||
if (NULL != (pos = strchr(data, '/')) && pos[1] == '/' && NULL != (pos = strchr(pos + 2, '/')))
|
||||
if (nullptr != (pos = strchr(data, '/')) && pos[1] == '/' && nullptr != (pos = strchr(pos + 2, '/')))
|
||||
{
|
||||
Pos query_string_or_fragment = strpbrk(pos, "?#");
|
||||
|
||||
@ -190,7 +190,7 @@ struct ExtractQueryString
|
||||
Pos pos = data;
|
||||
Pos end = pos + size;
|
||||
|
||||
if (NULL != (pos = strchr(data, '?')))
|
||||
if (nullptr != (pos = strchr(data, '?')))
|
||||
{
|
||||
Pos fragment = strchr(pos, '#');
|
||||
|
||||
@ -213,7 +213,7 @@ struct ExtractFragment
|
||||
Pos pos = data;
|
||||
Pos end = pos + size;
|
||||
|
||||
if (NULL != (pos = strchr(data, '#')))
|
||||
if (nullptr != (pos = strchr(data, '#')))
|
||||
{
|
||||
res_data = pos + (without_leading_char ? 1 : 0);
|
||||
res_size = end - res_data;
|
||||
@ -234,7 +234,7 @@ struct ExtractQueryStringAndFragment
|
||||
Pos pos = data;
|
||||
Pos end = pos + size;
|
||||
|
||||
if (NULL != (pos = strchr(data, '?')))
|
||||
if (nullptr != (pos = strchr(data, '?')))
|
||||
{
|
||||
res_data = pos + (without_leading_char ? 1 : 0);
|
||||
res_size = end - res_data;
|
||||
@ -291,32 +291,32 @@ struct ExtractURLParameterImpl
|
||||
{
|
||||
size_t cur_offset = offsets[i];
|
||||
|
||||
const char * pos = NULL;
|
||||
const char * pos = nullptr;
|
||||
|
||||
do
|
||||
{
|
||||
const char * str = reinterpret_cast<const char *>(&data[prev_offset]);
|
||||
|
||||
const char * begin = strchr(str, '?');
|
||||
if (begin == NULL)
|
||||
if (begin == nullptr)
|
||||
break;
|
||||
|
||||
pos = strstr(begin + 1, param_str);
|
||||
if (pos == NULL)
|
||||
if (pos == nullptr)
|
||||
break;
|
||||
if (pos != begin + 1 && *(pos - 1) != ';' && *(pos - 1) != '&')
|
||||
{
|
||||
pos = NULL;
|
||||
pos = nullptr;
|
||||
break;
|
||||
}
|
||||
|
||||
pos += param_len;
|
||||
} while (false);
|
||||
|
||||
if (pos != NULL)
|
||||
if (pos != nullptr)
|
||||
{
|
||||
const char * end = strpbrk(pos, "&;#");
|
||||
if (end == NULL)
|
||||
if (end == nullptr)
|
||||
end = pos + strlen(pos);
|
||||
|
||||
res_data.resize(res_offset + (end - pos) + 1);
|
||||
@ -367,15 +367,15 @@ struct CutURLParameterImpl
|
||||
do
|
||||
{
|
||||
const char * begin = strchr(url_begin, '?');
|
||||
if (begin == NULL)
|
||||
if (begin == nullptr)
|
||||
break;
|
||||
|
||||
const char * pos = strstr(begin + 1, param_str);
|
||||
if (pos == NULL)
|
||||
if (pos == nullptr)
|
||||
break;
|
||||
if (pos != begin + 1 && *(pos - 1) != ';' && *(pos - 1) != '&')
|
||||
{
|
||||
pos = NULL;
|
||||
pos = nullptr;
|
||||
break;
|
||||
}
|
||||
|
||||
@ -448,25 +448,25 @@ public:
|
||||
/// Получить следующий токен, если есть, или вернуть false.
|
||||
bool get(Pos & token_begin, Pos & token_end)
|
||||
{
|
||||
if (pos == NULL)
|
||||
if (pos == nullptr)
|
||||
return false;
|
||||
|
||||
if (first)
|
||||
{
|
||||
first = false;
|
||||
pos = strchr(pos, '?');
|
||||
if (pos == NULL)
|
||||
if (pos == nullptr)
|
||||
return false;
|
||||
++pos;
|
||||
}
|
||||
|
||||
token_begin = pos;
|
||||
pos = strchr(pos, '=');
|
||||
if (pos == NULL)
|
||||
if (pos == nullptr)
|
||||
return false;
|
||||
++pos;
|
||||
pos = strpbrk(pos, "&;#");
|
||||
if (pos == NULL)
|
||||
if (pos == nullptr)
|
||||
token_end = end;
|
||||
else
|
||||
token_end = pos++;
|
||||
@ -516,7 +516,7 @@ public:
|
||||
/// Получить следующий токен, если есть, или вернуть false.
|
||||
bool get(Pos & token_begin, Pos & token_end)
|
||||
{
|
||||
if (pos == NULL)
|
||||
if (pos == nullptr)
|
||||
return false;
|
||||
|
||||
if (first)
|
||||
@ -527,14 +527,14 @@ public:
|
||||
else
|
||||
pos = strchr(pos, '&');
|
||||
|
||||
if (pos == NULL)
|
||||
if (pos == nullptr)
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
token_begin = pos;
|
||||
|
||||
pos = strchr(pos, '=');
|
||||
if (pos == NULL)
|
||||
if (pos == nullptr)
|
||||
return false;
|
||||
else
|
||||
token_end = pos++;
|
||||
|
@ -48,7 +48,7 @@ public:
|
||||
/** Получить тип результата по типам аргументов и значениям константных аргументов.
|
||||
* Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
* Еще можно вернуть описание дополнительных столбцов, которые требуются для выполнения функции.
|
||||
* Для неконстантных столбцов arguments[i].column=NULL.
|
||||
* Для неконстантных столбцов arguments[i].column = nullptr.
|
||||
* Осмысленные типы элементов в out_prerequisites: APPLY_FUNCTION, ADD_COLUMN.
|
||||
*/
|
||||
virtual void getReturnTypeAndPrerequisites(
|
||||
|
@ -55,7 +55,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
AsynchronousWriteBuffer(WriteBuffer & out_) : WriteBuffer(NULL, 0), out(out_), memory(out.buffer().size()), pool(1), started(false)
|
||||
AsynchronousWriteBuffer(WriteBuffer & out_) : WriteBuffer(nullptr, 0), out(out_), memory(out.buffer().size()), pool(1), started(false)
|
||||
{
|
||||
/// Данные пишутся в дублирующий буфер.
|
||||
set(&memory[0], memory.size());
|
||||
|
@ -80,7 +80,7 @@ private:
|
||||
{
|
||||
if (!m_capacity)
|
||||
{
|
||||
m_data = NULL;
|
||||
m_data = nullptr;
|
||||
return;
|
||||
}
|
||||
|
||||
@ -121,7 +121,7 @@ protected:
|
||||
Memory memory;
|
||||
public:
|
||||
/// Если передать не-NULL existing_memory, то буфер не будет создавать свой кусок памяти, а будет использовать существующий (и не будет им владеть).
|
||||
BufferWithOwnMemory(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = NULL, size_t alignment = 0) : Base(NULL, 0), memory(existing_memory ? 0 : size, alignment)
|
||||
BufferWithOwnMemory(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) : Base(nullptr, 0), memory(existing_memory ? 0 : size, alignment)
|
||||
{
|
||||
Base::set(existing_memory ? existing_memory : &memory[0], size);
|
||||
}
|
||||
|
@ -67,7 +67,7 @@ private:
|
||||
|
||||
if (owned_cell->data.m_size == 0)
|
||||
{
|
||||
owned_cell = NULL;
|
||||
owned_cell = nullptr;
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -80,7 +80,7 @@ private:
|
||||
|
||||
public:
|
||||
CachedCompressedReadBuffer(const std::string & path_, UncompressedCache * cache_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: ReadBuffer(NULL, 0), path(path_), cache(cache_), buf_size(buf_size_), file_pos(0)
|
||||
: ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_), file_pos(0)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -72,7 +72,7 @@ protected:
|
||||
|
||||
public:
|
||||
ChunkedReadBuffer(ReadBuffer & in_, UInt64 assert_query_id_)
|
||||
: ReadBuffer(NULL, 0), in(in_), all_read(false), read_in_chunk(0), chunk_size(0), assert_query_id(assert_query_id_) {}
|
||||
: ReadBuffer(nullptr, 0), in(in_), all_read(false), read_in_chunk(0), chunk_size(0), assert_query_id(assert_query_id_) {}
|
||||
};
|
||||
|
||||
|
||||
|
@ -24,11 +24,10 @@ protected:
|
||||
ReadBuffer * compressed_in;
|
||||
|
||||
/// Если в буфере compressed_in помещается целый сжатый блок - используем его. Иначе - копируем данные по кусочкам в own_compressed_buffer.
|
||||
PODArray<char> own_compressed_buffer;
|
||||
char * compressed_buffer;
|
||||
size_t size_compressed;
|
||||
PODArray<char> own_compressed_buffer{QUICKLZ_HEADER_SIZE};
|
||||
char * compressed_buffer = nullptr;
|
||||
|
||||
qlz_state_decompress * qlz_state;
|
||||
qlz_state_decompress * qlz_state = nullptr;
|
||||
|
||||
/// Прочитать сжатые данные в compressed_buffer. Достать из их заголовка размер разжатых данных. Проверить чексумму.
|
||||
/// Возвращает количество прочитанных байт.
|
||||
@ -91,12 +90,8 @@ protected:
|
||||
|
||||
public:
|
||||
/// compressed_in можно инициализировать отложенно, но до первого вызова readCompressedData.
|
||||
CompressedReadBufferBase(ReadBuffer * in = NULL)
|
||||
:
|
||||
compressed_in(in),
|
||||
own_compressed_buffer(QUICKLZ_HEADER_SIZE),
|
||||
compressed_buffer(NULL),
|
||||
qlz_state(NULL)
|
||||
CompressedReadBufferBase(ReadBuffer * in = nullptr)
|
||||
: compressed_in(in)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -19,7 +19,7 @@ private:
|
||||
* - size_compressed содержит сжатый размер этого блока.
|
||||
*/
|
||||
ReadBufferFromFile file_in;
|
||||
size_t size_compressed;
|
||||
size_t size_compressed = 0;
|
||||
|
||||
bool nextImpl()
|
||||
{
|
||||
|
@ -36,7 +36,7 @@ private:
|
||||
|
||||
size_t uncompressed_size = offset();
|
||||
size_t compressed_size = 0;
|
||||
char * compressed_buffer_ptr = NULL;
|
||||
char * compressed_buffer_ptr = nullptr;
|
||||
|
||||
/** Для того, чтобы различить между QuickLZ и LZ4 и сохранить обратную совместимость (со случаем, когда использовался только QuickLZ),
|
||||
* используем старший бит первого байта в сжатых данных (который сейчас не используется в QuickLZ).
|
||||
|
@ -27,7 +27,7 @@ protected:
|
||||
/// Первое чтение
|
||||
if (working_buffer.size() == 0 && (*current)->position() != (*current)->buffer().end())
|
||||
{
|
||||
working_buffer = (*current)->buffer();
|
||||
working_buffer = Buffer((*current)->position(), (*current)->buffer().end());
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -46,14 +46,14 @@ protected:
|
||||
}
|
||||
}
|
||||
|
||||
working_buffer = (*current)->buffer();
|
||||
working_buffer = Buffer((*current)->position(), (*current)->buffer().end());
|
||||
return true;
|
||||
}
|
||||
|
||||
public:
|
||||
ConcatReadBuffer(const ReadBuffers & buffers_) : ReadBuffer(NULL, 0), buffers(buffers_), current(buffers.begin()) {}
|
||||
ConcatReadBuffer(const ReadBuffers & buffers_) : ReadBuffer(nullptr, 0), buffers(buffers_), current(buffers.begin()) {}
|
||||
|
||||
ConcatReadBuffer(ReadBuffer & buf1, ReadBuffer & buf2) : ReadBuffer(NULL, 0)
|
||||
ConcatReadBuffer(ReadBuffer & buf1, ReadBuffer & buf2) : ReadBuffer(nullptr, 0)
|
||||
{
|
||||
buffers.push_back(&buf1);
|
||||
buffers.push_back(&buf2);
|
||||
|
97
dbms/include/DB/IO/HashingWriteBuffer.h
Normal file
97
dbms/include/DB/IO/HashingWriteBuffer.h
Normal file
@ -0,0 +1,97 @@
|
||||
#pragma once
|
||||
|
||||
#include <city.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/IO/BufferWithOwnMemory.h>
|
||||
|
||||
#define DBMS_DEFAULT_HASHING_BLOCK_SIZE 2048ULL
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Вычисляет хеш от записываемых данных и передает их в указанный WriteBuffer.
|
||||
* В качестве основного буфера используется буфер вложенного WriteBuffer.
|
||||
*/
|
||||
class HashingWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
|
||||
{
|
||||
private:
|
||||
WriteBuffer & out;
|
||||
|
||||
size_t block_size;
|
||||
size_t block_pos;
|
||||
uint128 state;
|
||||
|
||||
void append(Position data)
|
||||
{
|
||||
state = CityHash128WithSeed(data, block_size, state);
|
||||
}
|
||||
|
||||
void nextImpl() override
|
||||
{
|
||||
size_t len = offset();
|
||||
|
||||
if (len)
|
||||
{
|
||||
Position data = working_buffer.begin();
|
||||
|
||||
if (block_pos + len < block_size)
|
||||
{
|
||||
memcpy(&memory[block_pos], data, len);
|
||||
block_pos += len;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (block_pos)
|
||||
{
|
||||
size_t n = block_size - block_pos;
|
||||
memcpy(&memory[block_pos], data, n);
|
||||
append(&memory[0]);
|
||||
len -= n;
|
||||
data += n;
|
||||
block_pos = 0;
|
||||
}
|
||||
|
||||
while (len >= block_size)
|
||||
{
|
||||
append(data);
|
||||
len -= block_size;
|
||||
data += block_size;
|
||||
}
|
||||
|
||||
if (len)
|
||||
{
|
||||
memcpy(&memory[0], data, len);
|
||||
block_pos = len;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
out.position() = pos;
|
||||
out.next();
|
||||
working_buffer = out.buffer();
|
||||
}
|
||||
|
||||
public:
|
||||
HashingWriteBuffer(
|
||||
WriteBuffer & out_,
|
||||
size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE)
|
||||
: BufferWithOwnMemory<WriteBuffer>(block_size_), out(out_), block_size(block_size_), block_pos(0)
|
||||
{
|
||||
out.next(); /// Если до нас в out что-то уже писали, не дадим остаткам этих данных повлиять на хеш.
|
||||
working_buffer = out.buffer();
|
||||
pos = working_buffer.begin();
|
||||
state = uint128(0, 0);
|
||||
}
|
||||
|
||||
uint128 getHash()
|
||||
{
|
||||
next();
|
||||
if (block_pos)
|
||||
return CityHash128WithSeed(&memory[0], block_pos, state);
|
||||
else
|
||||
return state;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -27,7 +27,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
LimitReadBuffer(ReadBuffer & in_, size_t limit_) : ReadBuffer(NULL, 0), in(in_), limit(limit_) {}
|
||||
LimitReadBuffer(ReadBuffer & in_, size_t limit_) : ReadBuffer(nullptr, 0), in(in_), limit(limit_) {}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -17,7 +17,7 @@ private:
|
||||
|
||||
public:
|
||||
ReadBufferFromFile(const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1,
|
||||
char * existing_memory = NULL, size_t alignment = 0)
|
||||
char * existing_memory = nullptr, size_t alignment = 0)
|
||||
: ReadBufferFromFileDescriptor(-1, buf_size, existing_memory, alignment), file_name(file_name_)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::FileOpen);
|
||||
|
@ -59,7 +59,7 @@ protected:
|
||||
}
|
||||
|
||||
public:
|
||||
ReadBufferFromFileDescriptor(int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = NULL, size_t alignment = 0)
|
||||
ReadBufferFromFileDescriptor(int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0)
|
||||
: BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment), fd(fd_), pos_in_file(0) {}
|
||||
|
||||
int getFD()
|
||||
|
96
dbms/include/DB/IO/ReadBufferFromHTTP.h
Normal file
96
dbms/include/DB/IO/ReadBufferFromHTTP.h
Normal file
@ -0,0 +1,96 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/URI.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Poco/Net/HTTPResponse.h>
|
||||
#include <Poco/Net/HTTPClientSession.h>
|
||||
|
||||
#include <DB/IO/ReadBuffer.h>
|
||||
#include <DB/IO/ReadBufferFromIStream.h>
|
||||
|
||||
#include <Yandex/logger_useful.h>
|
||||
|
||||
#define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Делает указанный HTTP-запрос и отдает ответ.
|
||||
*/
|
||||
class ReadBufferFromHTTP : public ReadBuffer
|
||||
{
|
||||
private:
|
||||
std::string host;
|
||||
int port;
|
||||
|
||||
Poco::Net::HTTPClientSession session;
|
||||
std::istream * istr; /// этим владеет session
|
||||
Poco::SharedPtr<ReadBufferFromIStream> impl;
|
||||
|
||||
public:
|
||||
typedef std::vector<std::pair<String, String> > Params;
|
||||
|
||||
ReadBufferFromHTTP(
|
||||
const std::string & host_,
|
||||
int port_,
|
||||
const Params & params,
|
||||
size_t timeout_ = 0,
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: ReadBuffer(nullptr, 0), host(host_), port(port_)
|
||||
{
|
||||
std::stringstream uri;
|
||||
uri << "http://" << host << ":" << port << "/";
|
||||
|
||||
bool first = true;
|
||||
for (const auto & it : params)
|
||||
{
|
||||
uri << (first ? "?" : "&");
|
||||
first = false;
|
||||
String encoded_key;
|
||||
String encoded_value;
|
||||
Poco::URI::encode(it.first, "=&#", encoded_key);
|
||||
Poco::URI::encode(it.second, "&#", encoded_value);
|
||||
uri << encoded_key << "=" << encoded_value;
|
||||
}
|
||||
|
||||
session.setHost(host);
|
||||
session.setPort(port);
|
||||
|
||||
/// устанавливаем таймаут
|
||||
session.setTimeout(Poco::Timespan(timeout_ ? timeout_ : DEFAULT_HTTP_READ_BUFFER_TIMEOUT, 0));
|
||||
|
||||
Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, uri.str());
|
||||
Poco::Net::HTTPResponse response;
|
||||
|
||||
LOG_TRACE((&Logger::get("ReadBufferFromHTTP")), "Sending request to " << uri.str());
|
||||
|
||||
session.sendRequest(request);
|
||||
istr = &session.receiveResponse(response);
|
||||
|
||||
Poco::Net::HTTPResponse::HTTPStatus status = response.getStatus();
|
||||
|
||||
if (status != Poco::Net::HTTPResponse::HTTP_OK)
|
||||
{
|
||||
std::stringstream error_message;
|
||||
error_message << "Received error from remote server " << uri.str() << ". HTTP status code: "
|
||||
<< status << ", body: " << istr->rdbuf();
|
||||
|
||||
throw Exception(error_message.str(), ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER);
|
||||
}
|
||||
|
||||
impl = new ReadBufferFromIStream(*istr, buffer_size_);
|
||||
}
|
||||
|
||||
bool nextImpl()
|
||||
{
|
||||
if (!impl->next())
|
||||
return false;
|
||||
internal_buffer = impl->buffer();
|
||||
working_buffer = internal_buffer;
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -18,6 +18,7 @@
|
||||
|
||||
#include <DB/IO/ReadBuffer.h>
|
||||
#include <DB/IO/VarInt.h>
|
||||
#include <city.h>
|
||||
|
||||
#define DEFAULT_MAX_STRING_SIZE 0x00FFFFFFULL
|
||||
|
||||
@ -114,6 +115,12 @@ inline void readChar(char & x, ReadBuffer & buf)
|
||||
}
|
||||
|
||||
void assertString(const char * s, ReadBuffer & buf);
|
||||
void assertEOF(ReadBuffer & buf);
|
||||
|
||||
inline void assertString(const String & s, ReadBuffer & buf)
|
||||
{
|
||||
assertString(s.c_str(), buf);
|
||||
}
|
||||
|
||||
|
||||
inline void readBoolText(bool & x, ReadBuffer & buf)
|
||||
@ -411,6 +418,7 @@ inline void readBinary(Float32 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(Float64 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(String & x, ReadBuffer & buf) { readStringBinary(x, buf); }
|
||||
inline void readBinary(bool & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(uint128 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
|
||||
inline void readBinary(VisitID_t & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
inline void readBinary(mysqlxx::Date & x, ReadBuffer & buf) { readPODBinary(x, buf); }
|
||||
|
@ -1,78 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/URI.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Poco/Net/HTTPResponse.h>
|
||||
#include <Poco/Net/HTTPClientSession.h>
|
||||
|
||||
#include <DB/IO/ReadBuffer.h>
|
||||
#include <DB/IO/ReadBufferFromIStream.h>
|
||||
|
||||
#include <Yandex/logger_useful.h>
|
||||
|
||||
#define DEFAULT_REMOTE_READ_BUFFER_TIMEOUT 1800
|
||||
#include <DB/IO/ReadBufferFromHTTP.h>
|
||||
#include "ReadHelpers.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Позволяет читать файл с удалённого сервера.
|
||||
/** Позволяет читать файл с удалённого сервера через riod.
|
||||
*/
|
||||
class RemoteReadBuffer : public ReadBuffer
|
||||
{
|
||||
private:
|
||||
std::string host;
|
||||
int port;
|
||||
std::string path;
|
||||
bool compress;
|
||||
|
||||
Poco::Net::HTTPClientSession session;
|
||||
std::istream * istr; /// этим владеет session
|
||||
Poco::SharedPtr<ReadBufferFromIStream> impl;
|
||||
Poco::SharedPtr<ReadBufferFromHTTP> impl;
|
||||
|
||||
public:
|
||||
RemoteReadBuffer(
|
||||
const std::string & host_,
|
||||
int port_,
|
||||
const std::string & path_,
|
||||
bool compress_ = true,
|
||||
size_t timeout_ = 0,
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: ReadBuffer(NULL, 0), host(host_), port(port_), path(path_), compress(compress_)
|
||||
const std::string & host,
|
||||
int port,
|
||||
const std::string & path,
|
||||
bool compress = true,
|
||||
size_t timeout = 0,
|
||||
size_t buffer_size = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: ReadBuffer(nullptr, 0)
|
||||
{
|
||||
std::string encoded_path;
|
||||
Poco::URI::encode(path, "&#", encoded_path);
|
||||
|
||||
std::stringstream uri;
|
||||
uri << "http://" << host << ":" << port << "/?action=read&path=" << encoded_path << "&compress=" << (compress ? "true" : "false");
|
||||
ReadBufferFromHTTP::Params params = {
|
||||
std::make_pair("action", "read"),
|
||||
std::make_pair("path", path),
|
||||
std::make_pair("compress", (compress ? "true" : "false"))};
|
||||
|
||||
session.setHost(host);
|
||||
session.setPort(port);
|
||||
|
||||
/// устанавливаем таймаут
|
||||
session.setTimeout(Poco::Timespan(timeout_ ? timeout_ : DEFAULT_REMOTE_READ_BUFFER_TIMEOUT, 0));
|
||||
|
||||
Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, uri.str());
|
||||
Poco::Net::HTTPResponse response;
|
||||
|
||||
LOG_TRACE((&Logger::get("RemoteReadBuffer")), "Sending request to " << uri.str());
|
||||
|
||||
session.sendRequest(request);
|
||||
istr = &session.receiveResponse(response);
|
||||
|
||||
Poco::Net::HTTPResponse::HTTPStatus status = response.getStatus();
|
||||
|
||||
if (status != Poco::Net::HTTPResponse::HTTP_OK)
|
||||
{
|
||||
std::stringstream error_message;
|
||||
error_message << "Received error from remote server " << uri.str() << ". HTTP status code: "
|
||||
<< status << ", body: " << istr->rdbuf();
|
||||
|
||||
throw Exception(error_message.str(), ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER);
|
||||
}
|
||||
|
||||
impl = new ReadBufferFromIStream(*istr, buffer_size_);
|
||||
impl = new ReadBufferFromHTTP(host, port, params, timeout, buffer_size);
|
||||
}
|
||||
|
||||
bool nextImpl()
|
||||
@ -89,42 +46,22 @@ public:
|
||||
const std::string & host,
|
||||
int port,
|
||||
const std::string & path,
|
||||
size_t timeout_ = 0)
|
||||
size_t timeout = 0)
|
||||
{
|
||||
std::string encoded_path;
|
||||
Poco::URI::encode(path, "&#", encoded_path);
|
||||
ReadBufferFromHTTP::Params params = {
|
||||
std::make_pair("action", "list"),
|
||||
std::make_pair("path", path)};
|
||||
|
||||
std::stringstream uri;
|
||||
uri << "http://" << host << ":" << port << "/?action=list&path=" << encoded_path;
|
||||
|
||||
Poco::Net::HTTPClientSession session;
|
||||
session.setHost(host);
|
||||
session.setPort(port);
|
||||
session.setTimeout(Poco::Timespan(timeout_ ? timeout_ : DEFAULT_REMOTE_READ_BUFFER_TIMEOUT, 0));
|
||||
|
||||
Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, uri.str());
|
||||
Poco::Net::HTTPResponse response;
|
||||
|
||||
LOG_TRACE((&Logger::get("RemoteReadBuffer")), "Sending request to " << uri.str());
|
||||
|
||||
session.sendRequest(request);
|
||||
std::istream * istr = &session.receiveResponse(response);
|
||||
|
||||
Poco::Net::HTTPResponse::HTTPStatus status = response.getStatus();
|
||||
|
||||
if (status != Poco::Net::HTTPResponse::HTTP_OK)
|
||||
{
|
||||
std::stringstream error_message;
|
||||
error_message << "Received error from remote server " << uri.str() << ". HTTP status code: "
|
||||
<< status << ", body: " << istr->rdbuf();
|
||||
|
||||
throw Exception(error_message.str(), ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER);
|
||||
}
|
||||
ReadBufferFromHTTP in(host, port, params, timeout);
|
||||
|
||||
std::vector<std::string> files;
|
||||
std::string s;
|
||||
while (getline(*istr, s, '\n') && !s.empty())
|
||||
while (!in.eof())
|
||||
{
|
||||
std::string s;
|
||||
readString(s, in);
|
||||
skipWhitespaceIfAny(in);
|
||||
files.push_back(s);
|
||||
}
|
||||
|
||||
return files;
|
||||
}
|
||||
|
@ -51,7 +51,7 @@ public:
|
||||
const std::string & tmp_path_ = "", const std::string & if_exists_ = "remove",
|
||||
bool decompress_ = false, size_t timeout_ = 0, unsigned connection_retries_ = 3,
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: WriteBuffer(NULL, 0), host(host_), port(port_), path(path_),
|
||||
: WriteBuffer(nullptr, 0), host(host_), port(port_), path(path_),
|
||||
tmp_path(tmp_path_), if_exists(if_exists_),
|
||||
decompress(decompress_), connection_retries(connection_retries_), finalized(false)
|
||||
{
|
||||
@ -231,7 +231,7 @@ private:
|
||||
{
|
||||
/// Если в прошлую попытку от сервера не пришло ответа, но файл всё же был переименован.
|
||||
if (i != 0 && e.code() == ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER
|
||||
&& NULL != strstr(e.displayText().data(), "File not found"))
|
||||
&& nullptr != strstr(e.displayText().data(), "File not found"))
|
||||
{
|
||||
LOG_TRACE((&Logger::get("RemoteWriteBuffer")), "File already renamed");
|
||||
}
|
||||
|
@ -17,17 +17,25 @@ struct UncompressedCacheCell
|
||||
size_t compressed_size;
|
||||
};
|
||||
|
||||
struct UncompressedSizeWeightFunction
|
||||
{
|
||||
size_t operator()(const UncompressedCacheCell & x) const
|
||||
{
|
||||
return x.data.size();
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Кэш разжатых блоков для CachedCompressedReadBuffer. thread-safe.
|
||||
*/
|
||||
class UncompressedCache : public LRUCache<UInt128, UncompressedCacheCell, UInt128TrivialHash>
|
||||
class UncompressedCache : public LRUCache<UInt128, UncompressedCacheCell, UInt128TrivialHash, UncompressedSizeWeightFunction>
|
||||
{
|
||||
private:
|
||||
typedef LRUCache<UInt128, UncompressedCacheCell, UInt128TrivialHash> Base;
|
||||
typedef LRUCache<UInt128, UncompressedCacheCell, UInt128TrivialHash, UncompressedSizeWeightFunction> Base;
|
||||
|
||||
public:
|
||||
UncompressedCache(size_t max_size_in_cells)
|
||||
: Base(max_size_in_cells) {}
|
||||
UncompressedCache(size_t max_size_in_bytes)
|
||||
: Base(max_size_in_bytes) {}
|
||||
|
||||
/// Посчитать ключ от пути к файлу и смещения.
|
||||
static UInt128 hash(const String & path_to_file, size_t offset)
|
||||
@ -53,6 +61,13 @@ public:
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void set(const Key & key, MappedPtr mapped)
|
||||
{
|
||||
Base::set(key, mapped);
|
||||
ProfileEvents::increment(ProfileEvents::UncompressedCacheWeightLost, current_weight_lost);
|
||||
current_weight_lost = 0;
|
||||
}
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<UncompressedCache> UncompressedCachePtr;
|
||||
|
@ -21,7 +21,7 @@ private:
|
||||
|
||||
public:
|
||||
WriteBufferFromFile(const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, mode_t mode = 0666,
|
||||
char * existing_memory = NULL, size_t alignment = 0)
|
||||
char * existing_memory = nullptr, size_t alignment = 0)
|
||||
: WriteBufferFromFileDescriptor(-1, buf_size, existing_memory, alignment), file_name(file_name_)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::FileOpen);
|
||||
|
@ -46,7 +46,7 @@ protected:
|
||||
}
|
||||
|
||||
public:
|
||||
WriteBufferFromFileDescriptor(int fd_ = -1, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = NULL, size_t alignment = 0)
|
||||
WriteBufferFromFileDescriptor(int fd_ = -1, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0)
|
||||
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment), fd(fd_) {}
|
||||
|
||||
/** Можно вызывать для инициализации, если нужный fd не был передан в конструктор.
|
||||
|
@ -24,7 +24,7 @@ class WriteBufferFromHTTPServerResponse : public BufferWithOwnMemory<WriteBuffer
|
||||
{
|
||||
private:
|
||||
Poco::Net::HTTPServerResponse & response;
|
||||
std::ostream * ostr;
|
||||
std::ostream * ostr = nullptr;
|
||||
|
||||
void nextImpl()
|
||||
{
|
||||
@ -43,7 +43,7 @@ private:
|
||||
|
||||
public:
|
||||
WriteBufferFromHTTPServerResponse(Poco::Net::HTTPServerResponse & response_, size_t size = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: BufferWithOwnMemory<WriteBuffer>(size), response(response_), ostr(NULL) {}
|
||||
: BufferWithOwnMemory<WriteBuffer>(size), response(response_) {}
|
||||
|
||||
/** Если данные ещё не были отправлены - отправить хотя бы HTTP заголовки.
|
||||
* Используйте эту функцию после того, как данные, возможно, были отправлены,
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <DB/IO/WriteIntText.h>
|
||||
#include <DB/IO/VarInt.h>
|
||||
#include <DB/IO/WriteBufferFromString.h>
|
||||
#include <city.h>
|
||||
|
||||
#define WRITE_HELPERS_DEFAULT_FLOAT_PRECISION 6U
|
||||
|
||||
@ -252,7 +253,7 @@ void writeAnyEscapedString(const String & s, WriteBuffer & buf)
|
||||
inline void writeEscapedString(const String & s, WriteBuffer & buf)
|
||||
{
|
||||
/// strpbrk в libc под Linux на процессорах с SSE 4.2 хорошо оптимизирована (этот if ускоряет код в 1.5 раза)
|
||||
if (NULL == strpbrk(s.data(), "\b\f\n\r\t\'\\") && strlen(s.data()) == s.size())
|
||||
if (nullptr == strpbrk(s.data(), "\b\f\n\r\t\'\\") && strlen(s.data()) == s.size())
|
||||
writeString(s, buf);
|
||||
else
|
||||
writeAnyEscapedString<'\''>(s, buf);
|
||||
@ -448,6 +449,7 @@ inline void writeBinary(const Float32 & x, WriteBuffer & buf) { writePODBinary(
|
||||
inline void writeBinary(const Float64 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
|
||||
inline void writeBinary(const String & x, WriteBuffer & buf) { writeStringBinary(x, buf); }
|
||||
inline void writeBinary(const bool & x, WriteBuffer & buf) { writePODBinary(x, buf); }
|
||||
inline void writeBinary(const uint128 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
|
||||
|
||||
inline void writeBinary(const VisitID_t & x, WriteBuffer & buf) { writePODBinary(static_cast<const UInt64 &>(x), buf); }
|
||||
inline void writeBinary(const mysqlxx::Date & x, WriteBuffer & buf) { writePODBinary(x, buf); }
|
||||
|
@ -1,5 +1,4 @@
|
||||
#ifndef DBMS_IO_COPY_DATA_H
|
||||
#define DBMS_IO_COPY_DATA_H
|
||||
#pragma once
|
||||
|
||||
#include <DB/IO/ReadBuffer.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
@ -12,6 +11,8 @@ namespace DB
|
||||
*/
|
||||
void copyData(ReadBuffer & from, WriteBuffer & to);
|
||||
|
||||
}
|
||||
/** Копирует bytes байт из ReadBuffer в WriteBuffer
|
||||
*/
|
||||
void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes);
|
||||
|
||||
#endif
|
||||
}
|
||||
|
@ -27,6 +27,7 @@ namespace DB
|
||||
struct AggregateDescription
|
||||
{
|
||||
AggregateFunctionPtr function;
|
||||
Array parameters; /// Параметры (параметрической) агрегатной функции.
|
||||
ColumnNumbers arguments;
|
||||
Names argument_names; /// Используются, если arguments не заданы.
|
||||
String column_name; /// Какое имя использовать для столбца со значениями агрегатной функции
|
||||
@ -59,7 +60,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
* - если при агрегации, до вызова Aggregator::convertToBlock вылетело исключение, то состояния агрегатных функций всё-равно должны быть уничтожены,
|
||||
* иначе для сложных состояний (наприемер, AggregateFunctionUniq), будут утечки памяти;
|
||||
* - чтобы, в этом случае, уничтожить состояния, в деструкторе вызывается метод Aggregator::destroyAggregateStates,
|
||||
* но только если переменная aggregator (см. ниже) не NULL;
|
||||
* но только если переменная aggregator (см. ниже) не nullptr;
|
||||
* - то есть, пока вы не передали владение состояниями агрегатных функций в ColumnAggregateFunction, установите переменную aggregator,
|
||||
* чтобы при возникновении исключения, состояния были корректно уничтожены.
|
||||
*
|
||||
@ -67,7 +68,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
* Но это вряд ли можно просто сделать, так как в этот же пул планируется класть строки переменной длины.
|
||||
* В этом случае, пул не сможет знать, по каким смещениям хранятся объекты.
|
||||
*/
|
||||
Aggregator * aggregator;
|
||||
Aggregator * aggregator = nullptr;
|
||||
|
||||
/// Пулы для состояний агрегатных функций. Владение потом будет передано в ColumnAggregateFunction.
|
||||
Arenas aggregates_pools;
|
||||
@ -75,7 +76,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
|
||||
/** Специализация для случая, когда ключи отсутствуют, и для ключей, не попавших в max_rows_to_group_by.
|
||||
*/
|
||||
AggregatedDataWithoutKey without_key;
|
||||
AggregatedDataWithoutKey without_key = nullptr;
|
||||
|
||||
/// Специализация для случая, когда есть один числовой ключ.
|
||||
/// auto_ptr - для ленивой инициализации (так как иначе HashMap в конструкторе выделяет и зануляет слишком много памяти).
|
||||
@ -106,9 +107,9 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
KEYS_128 = 4,
|
||||
HASHED = 5,
|
||||
};
|
||||
Type type;
|
||||
Type type = EMPTY;
|
||||
|
||||
AggregatedDataVariants() : aggregator(NULL), aggregates_pools(1, new Arena), aggregates_pool(&*aggregates_pools.back()), without_key(NULL), type(EMPTY) {}
|
||||
AggregatedDataVariants() : aggregates_pools(1, new Arena), aggregates_pool(&*aggregates_pools.back()) {}
|
||||
bool empty() const { return type == EMPTY; }
|
||||
|
||||
~AggregatedDataVariants();
|
||||
@ -137,10 +138,10 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
{
|
||||
case EMPTY: return 0;
|
||||
case WITHOUT_KEY: return 1;
|
||||
case KEY_64: return key64->size() + (without_key != NULL);
|
||||
case KEY_STRING: return key_string->size() + (without_key != NULL);
|
||||
case KEYS_128: return keys128->size() + (without_key != NULL);
|
||||
case HASHED: return hashed->size() + (without_key != NULL);
|
||||
case KEY_64: return key64->size() + (without_key != nullptr);
|
||||
case KEY_STRING: return key_string->size() + (without_key != nullptr);
|
||||
case KEYS_128: return keys128->size() + (without_key != nullptr);
|
||||
case HASHED: return hashed->size() + (without_key != nullptr);
|
||||
|
||||
default:
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
|
@ -86,10 +86,22 @@ private:
|
||||
* Элемент может остаться на месте, или переместиться в новое место "справа",
|
||||
* или переместиться левее по цепочке разрешения коллизий, из-за того, что элементы левее него были перемещены в новое место "справа".
|
||||
*/
|
||||
for (size_t i = 0; i < old_size; ++i)
|
||||
size_t i = 0;
|
||||
for (; i < old_size; ++i)
|
||||
if (buf[i].version == version)
|
||||
reinsert(buf[i]);
|
||||
|
||||
/** Также имеется особый случай:
|
||||
* если элемент должен был быть в конце старого буфера, [ x]
|
||||
* но находится в начале из-за цепочки разрешения коллизий, [o x]
|
||||
* то после ресайза, он сначала снова окажется не на своём месте, [ xo ]
|
||||
* и для того, чтобы перенести его куда надо,
|
||||
* надо будет после переноса всех элементов из старой половинки [ o x ]
|
||||
* обработать ещё хвостик из цепочки разрешения коллизий сразу после неё [ o x ]
|
||||
*/
|
||||
for (; buf[i].version == version; ++i)
|
||||
reinsert(buf[i]);
|
||||
|
||||
#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
|
||||
watch.stop();
|
||||
std::cerr << std::fixed << std::setprecision(3)
|
||||
|
@ -24,8 +24,10 @@
|
||||
#include <DB/Interpreters/Dictionaries.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/Interpreters/Cluster.h>
|
||||
#include <DB/Interpreters/InterserverIOHandler.h>
|
||||
#include <DB/Client/ConnectionPool.h>
|
||||
#include <statdaemons/ConfigProcessor.h>
|
||||
#include <zkutil/ZooKeeper.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -41,9 +43,6 @@ typedef std::map<String, StoragePtr> Tables;
|
||||
/// имя БД -> таблицы
|
||||
typedef std::map<String, Tables> Databases;
|
||||
|
||||
/// имя БД -> dropper
|
||||
typedef std::map<String, DatabaseDropperPtr> DatabaseDroppers;
|
||||
|
||||
/// (имя базы данных, имя таблицы)
|
||||
typedef std::pair<String, String> DatabaseAndTableName;
|
||||
|
||||
@ -51,7 +50,6 @@ typedef std::pair<String, String> DatabaseAndTableName;
|
||||
typedef std::map<DatabaseAndTableName, std::set<DatabaseAndTableName> > ViewDependencies;
|
||||
typedef std::vector<DatabaseAndTableName> Dependencies;
|
||||
|
||||
|
||||
/** Набор известных объектов, которые могут быть использованы в запросе.
|
||||
* Разделяемая часть. Порядок членов (порядок их уничтожения) очень важен.
|
||||
*/
|
||||
@ -74,9 +72,13 @@ struct ContextShared
|
||||
|
||||
mutable Poco::Mutex mutex; /// Для доступа и модификации разделяемых объектов.
|
||||
|
||||
mutable SharedPtr<zkutil::ZooKeeper> zookeeper; /// Клиент для ZooKeeper.
|
||||
|
||||
String interserver_io_host; /// Имя хоста по которым это сервер доступен для других серверов.
|
||||
int interserver_io_port; /// и порт,
|
||||
|
||||
String path; /// Путь к директории с данными, со слешем на конце.
|
||||
Databases databases; /// Список БД и таблиц в них.
|
||||
DatabaseDroppers database_droppers; /// Reference counter'ы для ленивого удаления БД.
|
||||
TableFunctionFactory table_function_factory; /// Табличные функции.
|
||||
FunctionFactory function_factory; /// Обычные функции.
|
||||
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
|
||||
@ -91,6 +93,8 @@ struct ContextShared
|
||||
ProcessList process_list; /// Исполняющиеся в данный момент запросы.
|
||||
ViewDependencies view_dependencies; /// Текущие зависимости
|
||||
ConfigurationPtr users_config; /// Конфиг с секциями users, profiles и quotas.
|
||||
InterserverIOHandler interserver_io_handler; /// Обработчик для межсерверной передачи данных.
|
||||
String default_replica_name; /// Имя реплики из конфига.
|
||||
|
||||
/// Кластеры для distributed таблиц
|
||||
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
|
||||
@ -137,7 +141,6 @@ struct ContextShared
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
database_droppers.clear();
|
||||
current_databases = databases;
|
||||
}
|
||||
|
||||
@ -163,27 +166,25 @@ class Context
|
||||
{
|
||||
private:
|
||||
typedef SharedPtr<ContextShared> Shared;
|
||||
Shared shared;
|
||||
Shared shared = new ContextShared;
|
||||
|
||||
String user; /// Текущий пользователь.
|
||||
Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос.
|
||||
QuotaForIntervalsPtr quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
|
||||
QuotaForIntervalsPtr quota = new QuotaForIntervals; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
|
||||
String current_database; /// Текущая БД.
|
||||
String current_query_id; /// Id текущего запроса.
|
||||
NamesAndTypesList columns; /// Столбцы текущей обрабатываемой таблицы.
|
||||
Settings settings; /// Настройки выполнения запроса.
|
||||
ProgressCallback progress_callback; /// Колбек для отслеживания прогресса выполнения запроса.
|
||||
ProcessList::Element * process_list_elem; /// Для отслеживания общего количества потраченных на запрос ресурсов.
|
||||
ProcessList::Element * process_list_elem = nullptr; /// Для отслеживания общего количества потраченных на запрос ресурсов.
|
||||
|
||||
String default_format; /// Формат, используемый, если сервер сам форматирует данные, и если в запросе не задан FORMAT.
|
||||
/// То есть, используется в HTTP-интерфейсе. Может быть не задан - тогда используется некоторый глобальный формат по-умолчанию.
|
||||
|
||||
Context * session_context; /// Контекст сессии или NULL, если его нет. (Возможно, равен this.)
|
||||
Context * global_context; /// Глобальный контекст или NULL, если его нет. (Возможно, равен this.)
|
||||
Tables external_tables; /// Временные таблицы.
|
||||
Context * session_context = nullptr; /// Контекст сессии или nullptr, если его нет. (Возможно, равен this.)
|
||||
Context * global_context = nullptr; /// Глобальный контекст или nullptr, если его нет. (Возможно, равен this.)
|
||||
|
||||
public:
|
||||
Context() : shared(new ContextShared), quota(new QuotaForIntervals), process_list_elem(NULL), session_context(NULL), global_context(NULL) {}
|
||||
|
||||
String getPath() const;
|
||||
void setPath(const String & path);
|
||||
|
||||
@ -214,8 +215,11 @@ public:
|
||||
void assertDatabaseExists(const String & database_name) const;
|
||||
void assertDatabaseDoesntExist(const String & database_name) const;
|
||||
|
||||
Tables getExternalTables() const;
|
||||
StoragePtr tryGetExternalTable(const String & table_name) const;
|
||||
StoragePtr getTable(const String & database_name, const String & table_name) const;
|
||||
StoragePtr tryGetTable(const String & database_name, const String & table_name) const;
|
||||
void addExternalTable(const String & table_name, StoragePtr storage);
|
||||
void addTable(const String & database_name, const String & table_name, StoragePtr table);
|
||||
void addDatabase(const String & database_name);
|
||||
|
||||
@ -232,8 +236,10 @@ public:
|
||||
String getDefaultFormat() const; /// Если default_format не задан - возвращается некоторый глобальный формат по-умолчанию.
|
||||
void setDefaultFormat(const String & name);
|
||||
|
||||
DatabaseDropperPtr getDatabaseDropper(const String & name);
|
||||
|
||||
/// Имя этой реплики из конфига.
|
||||
String getDefaultReplicaName() const;
|
||||
void setDefaultReplicaName(const String & name);
|
||||
|
||||
Settings getSettings() const;
|
||||
void setSettings(const Settings & settings_);
|
||||
|
||||
@ -253,6 +259,13 @@ public:
|
||||
const FormatFactory & getFormatFactory() const { return shared->format_factory; }
|
||||
const Dictionaries & getDictionaries() const;
|
||||
|
||||
InterserverIOHandler & getInterserverIOHandler() { return shared->interserver_io_handler; }
|
||||
|
||||
/// Как другие серверы могут обратиться к этому.
|
||||
void setInterserverIOHost(const String & host, int port);
|
||||
String getInterserverIOHost() const;
|
||||
int getInterserverIOPort() const;
|
||||
|
||||
/// Получить запрос на CREATE таблицы.
|
||||
ASTPtr getCreateQuery(const String & database_name, const String & table_name) const;
|
||||
|
||||
@ -285,7 +298,7 @@ public:
|
||||
* чтобы обновлять и контролировать информацию об общем количестве потраченных на запрос ресурсов.
|
||||
*/
|
||||
void setProcessListElement(ProcessList::Element * elem);
|
||||
/// Может вернуть NULL, если запрос не был вставлен в ProcessList.
|
||||
/// Может вернуть nullptr, если запрос не был вставлен в ProcessList.
|
||||
ProcessList::Element * getProcessListElement();
|
||||
|
||||
/// Список всех запросов.
|
||||
@ -293,13 +306,24 @@ public:
|
||||
const ProcessList & getProcessList() const { return shared->process_list; }
|
||||
|
||||
/// Создать кэш разжатых блоков указанного размера. Это можно сделать только один раз.
|
||||
void setUncompressedCache(size_t cache_size_in_cells);
|
||||
void setUncompressedCache(size_t max_size_in_bytes);
|
||||
UncompressedCachePtr getUncompressedCache() const;
|
||||
|
||||
void setZooKeeper(SharedPtr<zkutil::ZooKeeper> zookeeper);
|
||||
zkutil::ZooKeeper & getZooKeeper() const;
|
||||
|
||||
/// Создать кэш засечек указанного размера. Это можно сделать только один раз.
|
||||
void setMarkCache(size_t cache_size_in_bytes);
|
||||
MarkCachePtr getMarkCache() const;
|
||||
|
||||
/** Очистить кэши разжатых блоков и засечек.
|
||||
* Обычно это делается при переименовании таблиц, изменении типа столбцов, удалении таблицы.
|
||||
* - так как кэши привязаны к именам файлов, и становятся некорректными.
|
||||
* (при удалении таблицы - нужно, так как на её месте может появиться другая)
|
||||
* const - потому что изменение кэша не считается существенным.
|
||||
*/
|
||||
void resetCaches() const;
|
||||
|
||||
void initClusters();
|
||||
Cluster & getCluster(const std::string & cluster_name);
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
|
||||
#include <Yandex/MultiVersion.h>
|
||||
#include <Yandex/logger_useful.h>
|
||||
#include <statdaemons/RegionsHierarchy.h>
|
||||
#include <statdaemons/RegionsHierarchies.h>
|
||||
#include <statdaemons/TechDataHierarchy.h>
|
||||
#include <statdaemons/CategoriesHierarchy.h>
|
||||
#include <statdaemons/RegionsNames.h>
|
||||
@ -22,7 +22,7 @@ using Poco::SharedPtr;
|
||||
class Dictionaries
|
||||
{
|
||||
private:
|
||||
MultiVersion<RegionsHierarchy> regions_hierarchy;
|
||||
MultiVersion<RegionsHierarchies> regions_hierarchies;
|
||||
MultiVersion<TechDataHierarchy> tech_data_hierarchy;
|
||||
MultiVersion<CategoriesHierarchy> categories_hierarchy;
|
||||
MultiVersion<RegionsNames> regions_names;
|
||||
@ -81,9 +81,9 @@ private:
|
||||
|
||||
try
|
||||
{
|
||||
MultiVersion<RegionsHierarchy>::Version new_regions_hierarchy = new RegionsHierarchy;
|
||||
new_regions_hierarchy->reload();
|
||||
regions_hierarchy.set(new_regions_hierarchy);
|
||||
MultiVersion<RegionsHierarchies>::Version new_regions_hierarchies = new RegionsHierarchies;
|
||||
new_regions_hierarchies->reload();
|
||||
regions_hierarchies.set(new_regions_hierarchies);
|
||||
|
||||
}
|
||||
catch (...)
|
||||
@ -148,9 +148,9 @@ public:
|
||||
reloading_thread.join();
|
||||
}
|
||||
|
||||
MultiVersion<RegionsHierarchy>::Version getRegionsHierarchy() const
|
||||
MultiVersion<RegionsHierarchies>::Version getRegionsHierarchies() const
|
||||
{
|
||||
return regions_hierarchy.get();
|
||||
return regions_hierarchies.get();
|
||||
}
|
||||
|
||||
MultiVersion<TechDataHierarchy>::Version getTechDataHierarchy() const
|
||||
|
@ -139,7 +139,7 @@ public:
|
||||
{
|
||||
for (NamesAndTypesList::iterator it = input_columns.begin(); it != input_columns.end(); ++it)
|
||||
{
|
||||
sample_block.insert(ColumnWithNameAndType(NULL, it->second, it->first));
|
||||
sample_block.insert(ColumnWithNameAndType(nullptr, it->second, it->first));
|
||||
}
|
||||
}
|
||||
|
||||
@ -237,7 +237,7 @@ struct ExpressionActionsChain
|
||||
ExpressionActionsPtr actions;
|
||||
Names required_output;
|
||||
|
||||
Step(ExpressionActionsPtr actions_ = NULL, Names required_output_ = Names())
|
||||
Step(ExpressionActionsPtr actions_ = nullptr, Names required_output_ = Names())
|
||||
: actions(actions_), required_output(required_output_) {}
|
||||
};
|
||||
|
||||
|
@ -15,6 +15,8 @@ namespace DB
|
||||
{
|
||||
|
||||
/** Превращает выражение из синтаксического дерева в последовательность действий для его выполнения.
|
||||
*
|
||||
* NOTE: если ast - запрос SELECT из таблицы, структура этой таблицы не должна меняться во все время жизни ExpressionAnalyzer-а.
|
||||
*/
|
||||
class ExpressionAnalyzer : private boost::noncopyable
|
||||
{
|
||||
@ -51,7 +53,6 @@ public:
|
||||
Names getRequiredColumns();
|
||||
|
||||
/** Эти методы позволяют собрать цепочку преобразований над блоком, получающую значения в нужных секциях запроса.
|
||||
* Выполняют подзапросы в соответствующих частях запроса.
|
||||
*
|
||||
* Пример использования:
|
||||
* ExpressionActionsChain chain;
|
||||
@ -60,20 +61,25 @@ public:
|
||||
* analyzer.appendSelect(chain);
|
||||
* analyzer.appendOrderBy(chain);
|
||||
* chain.finalize();
|
||||
*
|
||||
* Если указано only_types=true, не выполняет подзапросы в соответствующих частях запроса. Полученные таким
|
||||
* образом действия не следует выполнять, они нужны только чтобы получить список столбцов с их типами.
|
||||
*/
|
||||
|
||||
void processGlobalOperations();
|
||||
|
||||
/// До агрегации:
|
||||
bool appendArrayJoin(ExpressionActionsChain & chain);
|
||||
bool appendWhere(ExpressionActionsChain & chain);
|
||||
bool appendGroupBy(ExpressionActionsChain & chain);
|
||||
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain);
|
||||
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
|
||||
|
||||
/// После агрегации:
|
||||
bool appendHaving(ExpressionActionsChain & chain);
|
||||
void appendSelect(ExpressionActionsChain & chain);
|
||||
bool appendOrderBy(ExpressionActionsChain & chain);
|
||||
bool appendHaving(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendSelect(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendOrderBy(ExpressionActionsChain & chain, bool only_types);
|
||||
/// Удаляет все столбцы кроме выбираемых SELECT, упорядочивает оставшиеся столбцы и переименовывает их в алиасы.
|
||||
void appendProjectResult(ExpressionActionsChain & chain);
|
||||
void appendProjectResult(ExpressionActionsChain & chain, bool only_types);
|
||||
|
||||
/// Если ast не запрос SELECT, просто получает все действия для вычисления выражения.
|
||||
/// Если project_result, в выходном блоке останутся только вычисленные значения в нужном порядке, переименованные в алиасы.
|
||||
@ -93,6 +99,11 @@ public:
|
||||
/// Если ast - запрос SELECT, получает имена (алиасы) и типы столбцов из секции SELECT.
|
||||
Block getSelectSampleBlock();
|
||||
|
||||
/// Все новые временные таблицы, полученные при выполнении подзапросов GLOBAL IN.
|
||||
Tables external_tables;
|
||||
|
||||
/// Создаем какие сможем Set из секции In для использования индекса по ним
|
||||
void makeSetsForIndex();
|
||||
private:
|
||||
typedef std::set<String> NamesSet;
|
||||
|
||||
@ -107,9 +118,7 @@ private:
|
||||
|
||||
/// Исходные столбцы.
|
||||
NamesAndTypesList columns;
|
||||
/// Столбцы после ARRAY JOIN. Если нет ARRAY JOIN, совпадает с columns.
|
||||
NamesAndTypesList columns_after_array_join;
|
||||
/// Столбцы после агрегации. Если нет агрегации, совпадает с columns_after_array_join.
|
||||
/// Столбцы после ARRAY JOIN и/или агрегации.
|
||||
NamesAndTypesList aggregated_columns;
|
||||
|
||||
/// Таблица, из которой делается запрос. Используется для sign-rewrite'а
|
||||
@ -122,7 +131,6 @@ private:
|
||||
AggregateDescriptions aggregate_descriptions;
|
||||
|
||||
std::map<std::string, SetPtr> sets_with_subqueries;
|
||||
|
||||
typedef std::map<String, ASTPtr> Aliases;
|
||||
Aliases aliases;
|
||||
|
||||
@ -178,7 +186,7 @@ private:
|
||||
|
||||
for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it)
|
||||
{
|
||||
all_columns.push_back(ColumnWithNameAndType(NULL, it->second, it->first));
|
||||
all_columns.push_back(ColumnWithNameAndType(nullptr, it->second, it->first));
|
||||
new_names.insert(it->first);
|
||||
stack.back().new_columns.insert(it->first);
|
||||
}
|
||||
@ -255,10 +263,17 @@ private:
|
||||
*/
|
||||
void normalizeTree();
|
||||
void normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, bool in_sign_rewritten);
|
||||
|
||||
|
||||
/// Обходит запрос и сохраняет найденные глобальные функции (например GLOBAL IN)
|
||||
void findGlobalFunctions(ASTPtr & ast, std::vector<ASTPtr> & global_nodes);
|
||||
void findExternalTables(ASTPtr & ast);
|
||||
|
||||
/// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn.
|
||||
void makeSet(ASTFunction * node, const Block & sample_block);
|
||||
|
||||
/// Выполнить подзапрос в секции GLOBAL IN и запомнить результат во временную таблицу типа memory
|
||||
/// Все новые временные таблицы хранятся в переменной external_tables
|
||||
void addExternalStorage(ASTFunction * node, size_t & name_id);
|
||||
|
||||
void getArrayJoinedColumns();
|
||||
void getArrayJoinedColumnsImpl(ASTPtr ast);
|
||||
void addMultipleArrayJoinAction(ExpressionActions & actions);
|
||||
@ -267,7 +282,7 @@ private:
|
||||
|
||||
void getRootActionsImpl(ASTPtr ast, bool no_subqueries, bool only_consts, ExpressionActions & actions);
|
||||
|
||||
void getActionsBeforeAggregationImpl(ASTPtr ast, ExpressionActions & actions);
|
||||
void getActionsBeforeAggregationImpl(ASTPtr ast, ExpressionActions & actions, bool no_subqueries);
|
||||
|
||||
/// Добавить агрегатные функции в aggregate_descriptions.
|
||||
/// Установить has_aggregation=true, если есть хоть одна агрегатная функция.
|
||||
@ -301,6 +316,9 @@ private:
|
||||
void assertSelect();
|
||||
void assertAggregation();
|
||||
void assertArrayJoin();
|
||||
|
||||
void makeExplicitSet(ASTFunction * node, const Block & sample_block, bool create_ordered_set);
|
||||
void makeSetsForIndexRecursively(ASTPtr & node, const Block & sample_block);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -181,10 +181,22 @@ private:
|
||||
* Элемент может остаться на месте, или переместиться в новое место "справа",
|
||||
* или переместиться левее по цепочке разрешения коллизий, из-за того, что элементы левее него были перемещены в новое место "справа".
|
||||
*/
|
||||
for (size_t i = 0; i < old_size; ++i)
|
||||
size_t i = 0;
|
||||
for (; i < old_size; ++i)
|
||||
if (!ZeroTraits::check(buf[i].first))
|
||||
reinsert(buf[i]);
|
||||
|
||||
/** Также имеется особый случай:
|
||||
* если элемент должен был быть в конце старого буфера, [ x]
|
||||
* но находится в начале из-за цепочки разрешения коллизий, [o x]
|
||||
* то после ресайза, он сначала снова окажется не на своём месте, [ xo ]
|
||||
* и для того, чтобы перенести его куда надо,
|
||||
* надо будет после переноса всех элементов из старой половинки [ o x ]
|
||||
* обработать ещё хвостик из цепочки разрешения коллизий сразу после неё [ o x ]
|
||||
*/
|
||||
for (; !ZeroTraits::check(buf[i].first); ++i)
|
||||
reinsert(buf[i]);
|
||||
|
||||
#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
|
||||
watch.stop();
|
||||
std::cerr << std::fixed << std::setprecision(3)
|
||||
|
@ -87,10 +87,22 @@ private:
|
||||
* Элемент может остаться на месте, или переместиться в новое место "справа",
|
||||
* или переместиться левее по цепочке разрешения коллизий, из-за того, что элементы левее него были перемещены в новое место "справа".
|
||||
*/
|
||||
for (size_t i = 0; i < old_size; ++i)
|
||||
size_t i = 0;
|
||||
for (; i < old_size; ++i)
|
||||
if (!ZeroTraits::check(buf[i]))
|
||||
reinsert(buf[i]);
|
||||
|
||||
/** Также имеется особый случай:
|
||||
* если элемент должен был быть в конце старого буфера, [ x]
|
||||
* но находится в начале из-за цепочки разрешения коллизий, [o x]
|
||||
* то после ресайза, он сначала снова окажется не на своём месте, [ xo ]
|
||||
* и для того, чтобы перенести его куда надо,
|
||||
* надо будет после переноса всех элементов из старой половинки [ o x ]
|
||||
* обработать ещё хвостик из цепочки разрешения коллизий сразу после неё [ o x ]
|
||||
*/
|
||||
for (; !ZeroTraits::check(buf[i]); ++i)
|
||||
reinsert(buf[i]);
|
||||
|
||||
#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
|
||||
watch.stop();
|
||||
std::cerr << std::fixed << std::setprecision(3)
|
||||
|
@ -5,6 +5,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTIdentifier;
|
||||
|
||||
/** Позволяет добавить или удалить столбец в таблице.
|
||||
*/
|
||||
@ -16,6 +17,8 @@ public:
|
||||
void execute();
|
||||
|
||||
private:
|
||||
void dropColumnFromAST(const ASTIdentifier & drop_column, ASTs & columns);
|
||||
|
||||
ASTPtr query_ptr;
|
||||
|
||||
Context context;
|
||||
|
@ -76,12 +76,9 @@ private:
|
||||
NamesAndTypesList columns;
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
|
||||
|
||||
if (!context.isTableExist(ast.database, ast.table))
|
||||
throw Exception("Table " + (ast.database.empty() ? "" : ast.database + ".") + ast.table + " doesn't exist", ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
columns = context.getTable(ast.database, ast.table)->getColumnsList();
|
||||
StoragePtr table = context.getTable(ast.database, ast.table);
|
||||
auto table_lock = table->lockStructure(false);
|
||||
columns = table->getColumnsList();
|
||||
}
|
||||
|
||||
ColumnString * name_column = new ColumnString;
|
||||
|
@ -18,6 +18,9 @@ public:
|
||||
/// Удаляет таблицу.
|
||||
void execute();
|
||||
|
||||
/// Удаляет таблицу, уже отцепленную от контекста (Context::detach).
|
||||
static void dropDetachedTable(String database_name, StoragePtr table, Context & context);
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
|
||||
@ -16,21 +17,23 @@ public:
|
||||
InterpreterInsertQuery(ASTPtr query_ptr_, Context & context_);
|
||||
|
||||
/** Выполнить запрос.
|
||||
* remaining_data_istr, если не NULL, может содержать нераспарсенные данные для вставки.
|
||||
* remaining_data_istr, если не nullptr, может содержать нераспарсенные данные для вставки.
|
||||
* (заранее может быть считан в оперативку для парсинга лишь небольшой кусок запроса, который содержит не все данные)
|
||||
*/
|
||||
void execute(ReadBuffer * remaining_data_istr);
|
||||
|
||||
/** Подготовить запрос к выполнению. Вернуть поток блоков, в который можно писать данные для выполнения запроса.
|
||||
* Или вернуть NULL, если запрос INSERT SELECT (самодостаточный запрос - не принимает входные данные).
|
||||
/** Подготовить запрос к выполнению. Вернуть потоки блоков
|
||||
* - поток, в который можно писать данные для выполнения запроса, если INSERT;
|
||||
* - поток, из которого можно читать результат выполнения запроса, если SELECT и подобные;
|
||||
* Или ничего, если запрос INSERT SELECT (самодостаточный запрос - не принимает входные данные, не отдаёт результат).
|
||||
*/
|
||||
BlockOutputStreamPtr execute();
|
||||
|
||||
Block getSampleBlock();
|
||||
BlockIO execute();
|
||||
|
||||
private:
|
||||
StoragePtr getTable();
|
||||
|
||||
Block getSampleBlock();
|
||||
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
};
|
||||
|
@ -20,7 +20,9 @@ public:
|
||||
void execute()
|
||||
{
|
||||
const ASTOptimizeQuery & ast = dynamic_cast<const ASTOptimizeQuery &>(*query_ptr);
|
||||
context.getTable(ast.database, ast.table)->optimize();
|
||||
StoragePtr table = context.getTable(ast.database, ast.table);
|
||||
auto table_lock = table->lockStructure(true);
|
||||
table->optimize();
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
*
|
||||
* ostr - куда писать результат выполнения запроса, если он есть.
|
||||
*
|
||||
* remaining_data_istr, если не NULL, может содержать нераспарсенный остаток запроса с данными.
|
||||
* remaining_data_istr, если не nullptr, может содержать нераспарсенный остаток запроса с данными.
|
||||
* (заранее может быть считан в оперативку для парсинга лишь небольшой кусок запроса, который содержит не все данные)
|
||||
*
|
||||
* В query_plan,
|
||||
|
@ -16,12 +16,12 @@ namespace DB
|
||||
class InterpreterSelectQuery
|
||||
{
|
||||
public:
|
||||
InterpreterSelectQuery(ASTPtr query_ptr_, const Context & context_, QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, size_t subquery_depth_ = 0, BlockInputStreamPtr input = NULL);
|
||||
InterpreterSelectQuery(ASTPtr query_ptr_, const Context & context_, QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, size_t subquery_depth_ = 0, BlockInputStreamPtr input = nullptr);
|
||||
|
||||
InterpreterSelectQuery(ASTPtr query_ptr_, const Context & context_, const Names & required_column_names,
|
||||
QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, size_t subquery_depth_ = 0, BlockInputStreamPtr input = NULL);
|
||||
QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, size_t subquery_depth_ = 0, BlockInputStreamPtr input = nullptr);
|
||||
|
||||
InterpreterSelectQuery(ASTPtr query_ptr_, const Context & context_, const Names & required_column_names, const NamesAndTypesList & table_column_names, QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, size_t subquery_depth_ = 0, BlockInputStreamPtr input = NULL);
|
||||
InterpreterSelectQuery(ASTPtr query_ptr_, const Context & context_, const Names & required_column_names, const NamesAndTypesList & table_column_names, QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, size_t subquery_depth_ = 0, BlockInputStreamPtr input = nullptr);
|
||||
|
||||
/// Выполнить запрос, получить поток блоков для чтения
|
||||
BlockInputStreamPtr execute();
|
||||
@ -34,10 +34,6 @@ public:
|
||||
DataTypes getReturnTypes();
|
||||
Block getSampleBlock();
|
||||
|
||||
/** Получить CREATE запрос для таблицы, из которой идёт выбор.
|
||||
*/
|
||||
ASTPtr getCreateQuery();
|
||||
|
||||
private:
|
||||
typedef Poco::SharedPtr<ExpressionAnalyzer> ExpressionAnalyzerPtr;
|
||||
|
||||
@ -47,8 +43,6 @@ private:
|
||||
*/
|
||||
void getDatabaseAndTableNames(String & database_name, String & table_name);
|
||||
|
||||
StoragePtr getTable();
|
||||
|
||||
/** Выбрать из списка столбцов какой-нибудь, лучше - минимального размера.
|
||||
*/
|
||||
String getAnyColumn();
|
||||
@ -71,7 +65,7 @@ private:
|
||||
void executeUnion( BlockInputStreams & streams);
|
||||
void executeLimit( BlockInputStreams & streams);
|
||||
void executeProjection( BlockInputStreams & streams, ExpressionActionsPtr expression);
|
||||
void executeDistinct( BlockInputStreams & streams, bool before_order);
|
||||
void executeDistinct( BlockInputStreams & streams, bool before_order, Names columns);
|
||||
void executeSubqueriesInSets( BlockInputStreams & streams, const Sets & sets);
|
||||
|
||||
|
||||
@ -83,7 +77,10 @@ private:
|
||||
size_t subquery_depth;
|
||||
ExpressionAnalyzerPtr query_analyzer;
|
||||
BlockInputStreams streams;
|
||||
StoragePtr table_function_storage;
|
||||
|
||||
/// Таблица, откуда читать данные, если не подзапрос.
|
||||
StoragePtr storage;
|
||||
IStorage::TableStructureReadLockPtr table_lock;
|
||||
|
||||
Logger * log;
|
||||
};
|
||||
|
@ -68,18 +68,9 @@ private:
|
||||
{
|
||||
const ASTShowCreateQuery & ast = dynamic_cast<const ASTShowCreateQuery &>(*query_ptr);
|
||||
|
||||
String res;
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
|
||||
|
||||
if (!context.isTableExist(ast.database, ast.table))
|
||||
throw Exception("Table " + (ast.database.empty() ? "" : ast.database + ".") + ast.table + " doesn't exist", ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
std::stringstream stream;
|
||||
formatAST(*context.getCreateQuery(ast.database, ast.table), stream, 0, false, true);
|
||||
res = stream.str();
|
||||
}
|
||||
std::stringstream stream;
|
||||
formatAST(*context.getCreateQuery(ast.database, ast.table), stream, 0, false, true);
|
||||
String res = stream.str();
|
||||
|
||||
ColumnWithNameAndType col;
|
||||
col.name = "statement";
|
||||
|
97
dbms/include/DB/Interpreters/InterserverIOHandler.h
Normal file
97
dbms/include/DB/Interpreters/InterserverIOHandler.h
Normal file
@ -0,0 +1,97 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/IO/ReadBuffer.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/Core/Types.h>
|
||||
#include <map>
|
||||
#include <Poco/Net/HTMLForm.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Обработчик запросов от других серверов.
|
||||
*/
|
||||
class InterserverIOEndpoint
|
||||
{
|
||||
public:
|
||||
virtual void processQuery(const Poco::Net::HTMLForm & params, WriteBuffer & out) = 0;
|
||||
|
||||
virtual ~InterserverIOEndpoint() {}
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<InterserverIOEndpoint> InterserverIOEndpointPtr;
|
||||
|
||||
|
||||
/** Сюда можно зарегистрировать сервис, обрататывающий запросы от других серверов.
|
||||
* Используется для передачи кусков в ReplicatedMergeTree.
|
||||
*/
|
||||
class InterserverIOHandler
|
||||
{
|
||||
public:
|
||||
void addEndpoint(const String & name, InterserverIOEndpointPtr endpoint)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
if (endpoint_map.count(name))
|
||||
throw Exception("Duplicate interserver IO endpoint: " + name, ErrorCodes::DUPLICATE_INTERSERVER_IO_ENDPOINT);
|
||||
endpoint_map[name] = endpoint;
|
||||
}
|
||||
|
||||
void removeEndpoint(const String & name)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
if (!endpoint_map.count(name))
|
||||
throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
|
||||
endpoint_map.erase(name);
|
||||
}
|
||||
|
||||
InterserverIOEndpointPtr getEndpoint(const String & name)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
if (!endpoint_map.count(name))
|
||||
throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
|
||||
return endpoint_map[name];
|
||||
}
|
||||
|
||||
private:
|
||||
typedef std::map<String, InterserverIOEndpointPtr> EndpointMap;
|
||||
|
||||
EndpointMap endpoint_map;
|
||||
Poco::FastMutex mutex;
|
||||
};
|
||||
|
||||
/// В конструкторе вызывает addEndpoint, в деструкторе - removeEndpoint.
|
||||
class InterserverIOEndpointHolder
|
||||
{
|
||||
public:
|
||||
InterserverIOEndpointHolder(const String & name_, InterserverIOEndpointPtr endpoint_, InterserverIOHandler & handler_)
|
||||
: name(name_), endpoint(endpoint_), handler(handler_)
|
||||
{
|
||||
handler.addEndpoint(name, endpoint);
|
||||
}
|
||||
|
||||
InterserverIOEndpointPtr getEndpoint()
|
||||
{
|
||||
return endpoint;
|
||||
}
|
||||
|
||||
~InterserverIOEndpointHolder()
|
||||
{
|
||||
try
|
||||
{
|
||||
handler.removeEndpoint(name);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("~InterserverIOEndpointHolder");
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
String name;
|
||||
InterserverIOEndpointPtr endpoint;
|
||||
InterserverIOHandler & handler;
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<InterserverIOEndpointHolder> InterserverIOEndpointHolderPtr;
|
||||
|
||||
}
|
@ -21,7 +21,7 @@ struct Limits
|
||||
* По-умолчанию: всё не ограничено, кроме довольно слабых ограничений на глубину рекурсии и размер выражений.
|
||||
*/
|
||||
|
||||
#define APPLY_FOR_SETTINGS(M) \
|
||||
#define APPLY_FOR_LIMITS(M) \
|
||||
/** Ограничения на чтение из самых "глубоких" источников. \
|
||||
* То есть, только в самом глубоком подзапросе. \
|
||||
* При чтении с удалённого сервера, проверяется только на удалённом сервере. \
|
||||
@ -77,7 +77,7 @@ struct Limits
|
||||
#define DECLARE(TYPE, NAME, DEFAULT) \
|
||||
TYPE NAME {DEFAULT};
|
||||
|
||||
APPLY_FOR_SETTINGS(DECLARE)
|
||||
APPLY_FOR_LIMITS(DECLARE)
|
||||
|
||||
#undef DECLARE
|
||||
|
||||
@ -88,7 +88,7 @@ struct Limits
|
||||
else if (name == #NAME) NAME.set(value);
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_SETTINGS(TRY_SET)
|
||||
APPLY_FOR_LIMITS(TRY_SET)
|
||||
else
|
||||
return false;
|
||||
|
||||
@ -104,7 +104,7 @@ struct Limits
|
||||
else if (name == #NAME) NAME.set(buf);
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_SETTINGS(TRY_SET)
|
||||
APPLY_FOR_LIMITS(TRY_SET)
|
||||
else
|
||||
return false;
|
||||
|
||||
@ -121,7 +121,7 @@ struct Limits
|
||||
else if (name == #NAME) NAME.set(value);
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_SETTINGS(TRY_SET)
|
||||
APPLY_FOR_LIMITS(TRY_SET)
|
||||
else
|
||||
return false;
|
||||
|
||||
@ -143,12 +143,10 @@ private:
|
||||
NAME.write(buf); \
|
||||
}
|
||||
|
||||
APPLY_FOR_SETTINGS(WRITE)
|
||||
APPLY_FOR_LIMITS(WRITE)
|
||||
|
||||
#undef WRITE
|
||||
}
|
||||
|
||||
#undef APPLY_FOR_SETTINGS
|
||||
};
|
||||
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <set>
|
||||
#include <boost/concept_check.hpp>
|
||||
|
||||
#include <statdaemons/Stopwatch.h>
|
||||
|
||||
@ -19,6 +20,8 @@
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
|
||||
#include <DB/Storages/MergeTree/BoolMask.h>
|
||||
#include <DB/Storages/MergeTree/PKCondition.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -42,8 +45,9 @@ public:
|
||||
/** Создать множество по выражению (для перечисления в самом запросе).
|
||||
* types - типы того, что стоит слева от IN.
|
||||
* node - это список значений: 1, 2, 3 или список tuple-ов: (1, 2), (3, 4), (5, 6).
|
||||
* create_ordered_set - создавать ли вектор упорядоченных элементов. Нужен для работы индекса
|
||||
*/
|
||||
void createFromAST(DataTypes & types, ASTPtr node);
|
||||
void createFromAST(DataTypes & types, ASTPtr node, bool create_ordered_set);
|
||||
|
||||
/** Запомнить поток блоков (для подзапросов), чтобы потом его можно было прочитать и создать множество.
|
||||
*/
|
||||
@ -52,7 +56,7 @@ public:
|
||||
BlockInputStreamPtr getSource() { return source; }
|
||||
|
||||
// Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
|
||||
bool insertFromBlock(Block & block);
|
||||
bool insertFromBlock(Block & block, bool create_ordered_set = false);
|
||||
|
||||
size_t size() const { return getTotalRowCount(); }
|
||||
|
||||
@ -60,6 +64,30 @@ public:
|
||||
* Записать результат в столбец в позиции result.
|
||||
*/
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result, bool negative) const;
|
||||
|
||||
std::string describe()
|
||||
{
|
||||
if (!ordered_set_elements)
|
||||
return "{}";
|
||||
|
||||
bool first = true;
|
||||
std::stringstream ss;
|
||||
|
||||
ss << "{";
|
||||
for (const Field & f : *ordered_set_elements)
|
||||
{
|
||||
if (!first)
|
||||
ss << ", " << f;
|
||||
else
|
||||
ss << f;
|
||||
first = false;
|
||||
}
|
||||
ss << "}";
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
/// проверяет есть ли в Set элементы для заданного диапазона индекса
|
||||
BoolMask mayBeTrueInRange(const Range & range);
|
||||
|
||||
private:
|
||||
/** Разные структуры данных, которые могут использоваться для проверки принадлежности
|
||||
@ -125,9 +153,15 @@ private:
|
||||
size_t getTotalRowCount() const;
|
||||
/// Считает суммарный размер в байтах буфферов всех Set'ов + размер string_pool'а
|
||||
size_t getTotalByteCount() const;
|
||||
|
||||
/// вектор упорядоченных элементов Set
|
||||
/// нужен для работы индекса по первичному ключу в секции In
|
||||
typedef std::vector<Field> OrderedSetElements;
|
||||
typedef std::unique_ptr<OrderedSetElements> OrderedSetElementsPtr;
|
||||
OrderedSetElementsPtr ordered_set_elements;
|
||||
};
|
||||
|
||||
typedef SharedPtr<Set> SetPtr;
|
||||
typedef Poco::SharedPtr<Set> SetPtr;
|
||||
typedef std::vector<SetPtr> Sets;
|
||||
|
||||
|
||||
|
@ -24,6 +24,13 @@ struct Settings
|
||||
*/
|
||||
|
||||
#define APPLY_FOR_SETTINGS(M) \
|
||||
/** При записи данных, для сжатия выделяется буфер размером max_compress_block_size. При переполнении буфера или если в буфер */ \
|
||||
/** записано данных больше или равно, чем min_compress_block_size, то при очередной засечке, данные так же будут сжиматься */ \
|
||||
/** В результате, для маленьких столбцов (числа 1-8 байт), при index_granularity = 8192, размер блока будет 64 KБ. */ \
|
||||
/** А для больших столбцов (Title - строка ~100 байт), размер блока будет ~819 КБ. */ \
|
||||
/** За счёт этого, коэффициент сжатия почти не ухудшится. */ \
|
||||
M(SettingUInt64, min_compress_block_size, DEFAULT_MIN_COMPRESS_BLOCK_SIZE) \
|
||||
M(SettingUInt64, max_compress_block_size, DEFAULT_MAX_COMPRESS_BLOCK_SIZE) \
|
||||
/** Максимальный размер блока для чтения */ \
|
||||
M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE) \
|
||||
/** Максимальное количество потоков выполнения запроса */ \
|
||||
@ -94,7 +101,8 @@ struct Settings
|
||||
void setProfile(const String & profile_name, Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
/// Прочитать настройки из буфера. Они записаны как набор name-value пар, идущих подряд, заканчивающихся пустым name.
|
||||
void deserialize(ReadBuffer & buf);
|
||||
/// Если выставлен флаг check_readonly, в настройках выставлено readonly, но пришли какие-то изменения кинуть исключение.
|
||||
void deserialize(ReadBuffer & buf, bool check_readonly = false);
|
||||
|
||||
/// Записать изменённые настройки в буфер. (Например, для отправки на удалённый сервер.)
|
||||
void serialize(WriteBuffer & buf) const;
|
||||
|
@ -52,7 +52,7 @@ public:
|
||||
{
|
||||
const char * pos = strchr(str.c_str(), '/');
|
||||
|
||||
if (NULL == pos)
|
||||
if (nullptr == pos)
|
||||
{
|
||||
construct(Poco::Net::IPAddress(str));
|
||||
}
|
||||
@ -111,20 +111,20 @@ public:
|
||||
Poco::Net::IPAddress addr_v6 = toIPv6(addr);
|
||||
|
||||
/// Резолвим вручную, потому что в Poco не используется флаг AI_ALL, а он важен.
|
||||
addrinfo * ai = NULL;
|
||||
addrinfo * ai = nullptr;
|
||||
|
||||
addrinfo hints;
|
||||
memset(&hints, 0, sizeof(hints));
|
||||
hints.ai_family = AF_UNSPEC;
|
||||
hints.ai_flags |= AI_V4MAPPED | AI_ALL;
|
||||
|
||||
int ret = getaddrinfo(host.c_str(), NULL, &hints, &ai);
|
||||
int ret = getaddrinfo(host.c_str(), nullptr, &hints, &ai);
|
||||
if (0 != ret)
|
||||
throw Exception("Cannot getaddrinfo: " + std::string(gai_strerror(ret)), ErrorCodes::DNS_ERROR);
|
||||
|
||||
try
|
||||
{
|
||||
for (; ai != NULL; ai = ai->ai_next)
|
||||
for (; ai != nullptr; ai = ai->ai_next)
|
||||
{
|
||||
if (ai->ai_addrlen && ai->ai_addr)
|
||||
{
|
||||
@ -175,7 +175,7 @@ public:
|
||||
|
||||
/// Резолвим вручную, потому что в Poco нет такой функциональности.
|
||||
char domain[1024];
|
||||
int gai_errno = getnameinfo(sock_addr.addr(), sock_addr.length(), domain, sizeof(domain), NULL, 0, NI_NAMEREQD);
|
||||
int gai_errno = getnameinfo(sock_addr.addr(), sock_addr.length(), domain, sizeof(domain), nullptr, 0, NI_NAMEREQD);
|
||||
if (0 != gai_errno)
|
||||
throw Exception("Cannot getnameinfo: " + std::string(gai_strerror(gai_errno)), ErrorCodes::DNS_ERROR);
|
||||
|
||||
|
@ -54,7 +54,7 @@ public:
|
||||
String table;
|
||||
|
||||
|
||||
ASTAlterQuery(StringRange range_ = StringRange(NULL, NULL)) : IAST(range_) {};
|
||||
ASTAlterQuery(StringRange range_ = StringRange()) : IAST(range_) {};
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const { return ("AlterQuery_" + database + "_" + table); };
|
||||
|
@ -18,6 +18,7 @@ public:
|
||||
bool is_view;
|
||||
bool is_materialized_view;
|
||||
bool is_populate;
|
||||
bool is_temporary;
|
||||
String database;
|
||||
String table;
|
||||
ASTPtr columns;
|
||||
@ -27,8 +28,8 @@ public:
|
||||
String as_table;
|
||||
ASTPtr select;
|
||||
|
||||
ASTCreateQuery() : attach(false), if_not_exists(false), is_view(false), is_materialized_view(false), is_populate(false) {}
|
||||
ASTCreateQuery(StringRange range_) : IAST(range_), attach(false), if_not_exists(false), is_view(false), is_materialized_view(false),is_populate(false) {}
|
||||
ASTCreateQuery() : attach(false), if_not_exists(false), is_view(false), is_materialized_view(false), is_populate(false), is_temporary(false) {}
|
||||
ASTCreateQuery(StringRange range_) : IAST(range_), attach(false), if_not_exists(false), is_view(false), is_materialized_view(false), is_populate(false), is_temporary(false) {}
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const { return (attach ? "AttachQuery_" : "CreateQuery_") + database + "_" + table; };
|
||||
|
@ -18,12 +18,14 @@ public:
|
||||
ASTPtr columns;
|
||||
String format;
|
||||
ASTPtr select;
|
||||
/// Идентификатор запроса INSERT. Используется при репликации.
|
||||
String insert_id;
|
||||
/// Данные для вставки
|
||||
const char * data;
|
||||
const char * end;
|
||||
const char * data = nullptr;
|
||||
const char * end = nullptr;
|
||||
|
||||
ASTInsertQuery() {}
|
||||
ASTInsertQuery(StringRange range_) : IAST(range_), data(NULL), end(NULL) {}
|
||||
ASTInsertQuery(StringRange range_) : IAST(range_) {}
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const { return "InsertQuery_" + database + "_" + table; };
|
||||
|
@ -17,12 +17,12 @@ public:
|
||||
int direction; /// 1, если ASC, -1, если DESC
|
||||
|
||||
/** Collator для locale-specific сортировки строк.
|
||||
* Если NULL, то производится сортировка по байтам.
|
||||
* Если nullptr, то производится сортировка по байтам.
|
||||
*/
|
||||
Poco::SharedPtr<Collator> collator;
|
||||
|
||||
ASTOrderByElement() {}
|
||||
ASTOrderByElement(StringRange range_, int direction_, const Poco::SharedPtr<Collator> & collator_ = NULL)
|
||||
ASTOrderByElement(StringRange range_, int direction_, const Poco::SharedPtr<Collator> & collator_ = nullptr)
|
||||
: IAST(range_), direction(direction_), collator(collator_) {}
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
|
@ -37,12 +37,36 @@ public:
|
||||
String getID() const { return "SelectQuery"; };
|
||||
|
||||
|
||||
static bool hasArrayJoin(const ASTPtr & ast)
|
||||
{
|
||||
if (const ASTFunction * function = dynamic_cast<const ASTFunction *>(&*ast))
|
||||
{
|
||||
if (function->kind == ASTFunction::ARRAY_JOIN)
|
||||
return true;
|
||||
}
|
||||
for (const auto & child : ast->children)
|
||||
if (hasArrayJoin(child))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Переписывает select_expression_list, чтобы вернуть только необходимые столбцы в правильном порядке.
|
||||
void rewriteSelectExpressionList(const Names & column_names)
|
||||
{
|
||||
ASTPtr result = new ASTExpressionList;
|
||||
ASTs asts = select_expression_list->children;
|
||||
|
||||
/// Не будем выбрасывать выражения, содержащие функцию arrayJoin.
|
||||
std::set<ASTPtr> unremovable_asts;
|
||||
for (size_t j = 0; j < asts.size(); ++j)
|
||||
{
|
||||
if (hasArrayJoin(asts[j]))
|
||||
{
|
||||
result->children.push_back(asts[j]->clone());
|
||||
unremovable_asts.insert(asts[j]);
|
||||
}
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < column_names.size(); ++i)
|
||||
{
|
||||
bool done = 0;
|
||||
@ -50,7 +74,8 @@ public:
|
||||
{
|
||||
if (asts[j]->getAlias() == column_names[i])
|
||||
{
|
||||
result->children.push_back(asts[j]->clone());
|
||||
if (!unremovable_asts.count(asts[j]))
|
||||
result->children.push_back(asts[j]->clone());
|
||||
done = 1;
|
||||
}
|
||||
}
|
||||
@ -60,7 +85,21 @@ public:
|
||||
DB::ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
|
||||
}
|
||||
select_expression_list.swap(result);
|
||||
|
||||
for (auto & child : children)
|
||||
{
|
||||
if (child == select_expression_list)
|
||||
{
|
||||
child = result;
|
||||
break;
|
||||
}
|
||||
}
|
||||
select_expression_list = result;
|
||||
|
||||
/** NOTE: Может показаться, что мы могли испортить запрос, выбросив выражение с алиасом, который используется где-то еще.
|
||||
* Такого произойти не может, потому что этот метод вызывается всегда для запроса, на котором хоть раз создавали
|
||||
* ExpressionAnalyzer, что гарантирует, что в нем все алиасы уже подставлены. Не совсем очевидная логика :)
|
||||
*/
|
||||
}
|
||||
|
||||
ASTPtr clone() const
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user