mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 18:12:02 +00:00
Merge
This commit is contained in:
commit
eb607dc611
@ -174,6 +174,11 @@ public:
|
||||
*/
|
||||
void disconnect();
|
||||
|
||||
/** Заполнить информацию, которая необходима при получении блока для некоторых задач
|
||||
* (пока только для запроса DESCRIBE TABLE с Distributed-таблицами).
|
||||
*/
|
||||
void fillBlockExtraInfo(BlockExtraInfo & info) const;
|
||||
|
||||
size_t outBytesCount() const { return !out.isNull() ? out->count() : 0; }
|
||||
size_t inBytesCount() const { return !in.isNull() ? in->count() : 0; }
|
||||
|
||||
|
@ -26,18 +26,33 @@ class IConnectionPool : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
typedef PoolBase<Connection>::Entry Entry;
|
||||
virtual Entry get(const Settings * settings = nullptr) = 0;
|
||||
|
||||
public:
|
||||
virtual ~IConnectionPool() {}
|
||||
|
||||
/** Выделяет соединение для работы. */
|
||||
Entry get(const Settings * settings = nullptr)
|
||||
{
|
||||
return doGet(settings);
|
||||
}
|
||||
|
||||
/** Выделяет до указанного количества соединений для работы.
|
||||
* Соединения предоставляют доступ к разным репликам одного шарда.
|
||||
* Если флаг get_all установлен, все соединения достаются.
|
||||
* Выкидывает исключение, если не удалось выделить ни одного соединения.
|
||||
*/
|
||||
virtual std::vector<Entry> getMany(const Settings * settings = nullptr)
|
||||
std::vector<Entry> getMany(const Settings * settings = nullptr, bool get_all = false)
|
||||
{
|
||||
return doGetMany(settings, get_all);
|
||||
}
|
||||
|
||||
protected:
|
||||
virtual Entry doGet(const Settings * settings) = 0;
|
||||
|
||||
virtual std::vector<Entry> doGetMany(const Settings * settings, bool get_all)
|
||||
{
|
||||
return std::vector<Entry>{ get(settings) };
|
||||
}
|
||||
|
||||
virtual ~IConnectionPool() {}
|
||||
};
|
||||
|
||||
typedef SharedPtr<IConnectionPool> ConnectionPoolPtr;
|
||||
@ -87,16 +102,6 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
/** Выделяет соединение для работы. */
|
||||
Entry get(const Settings * settings = nullptr) override
|
||||
{
|
||||
if (settings)
|
||||
return Base::get(settings->queue_max_wait_ms.totalMilliseconds());
|
||||
else
|
||||
return Base::get(-1);
|
||||
}
|
||||
|
||||
const std::string & getHost() const
|
||||
{
|
||||
return host;
|
||||
@ -113,6 +118,15 @@ protected:
|
||||
connect_timeout, receive_timeout, send_timeout);
|
||||
}
|
||||
|
||||
private:
|
||||
Entry doGet(const Settings * settings) override
|
||||
{
|
||||
if (settings)
|
||||
return Base::get(settings->queue_max_wait_ms.totalMilliseconds());
|
||||
else
|
||||
return Base::get(-1);
|
||||
}
|
||||
|
||||
private:
|
||||
String host;
|
||||
UInt16 port;
|
||||
|
@ -51,22 +51,6 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
/** Выделяет соединение для работы. */
|
||||
Entry get(const Settings * settings = nullptr) override
|
||||
{
|
||||
applyLoadBalancing(settings);
|
||||
return Base::get(settings);
|
||||
}
|
||||
|
||||
/** Выделяет до указанного количества соединений для работы.
|
||||
* Соединения предоставляют доступ к разным репликам одного шарда.
|
||||
*/
|
||||
std::vector<Entry> getMany(const Settings * settings = nullptr) override
|
||||
{
|
||||
applyLoadBalancing(settings);
|
||||
return Base::getMany(settings);
|
||||
}
|
||||
|
||||
protected:
|
||||
bool tryGet(ConnectionPoolPtr pool, const Settings * settings, Entry & out_entry, std::stringstream & fail_message) override
|
||||
{
|
||||
@ -87,8 +71,21 @@ protected:
|
||||
}
|
||||
|
||||
private:
|
||||
std::vector<size_t> hostname_differences; /// Расстояния от имени этого хоста до имен хостов пулов.
|
||||
LoadBalancing default_load_balancing;
|
||||
/** Выделяет соединение для работы. */
|
||||
Entry doGet(const Settings * settings) override
|
||||
{
|
||||
applyLoadBalancing(settings);
|
||||
return Base::get(settings);
|
||||
}
|
||||
|
||||
/** Выделяет до указанного количества соединений для работы.
|
||||
* Соединения предоставляют доступ к разным репликам одного шарда.
|
||||
*/
|
||||
std::vector<Entry> doGetMany(const Settings * settings, bool get_all) override
|
||||
{
|
||||
applyLoadBalancing(settings);
|
||||
return Base::getMany(settings, get_all);
|
||||
}
|
||||
|
||||
void applyLoadBalancing(const Settings * settings)
|
||||
{
|
||||
@ -108,6 +105,10 @@ private:
|
||||
throw Exception("Unknown load_balancing_mode: " + toString(static_cast<int>(load_balancing)), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
std::vector<size_t> hostname_differences; /// Расстояния от имени этого хоста до имен хостов пулов.
|
||||
LoadBalancing default_load_balancing;
|
||||
};
|
||||
|
||||
|
||||
|
@ -23,8 +23,13 @@ public:
|
||||
/// Принимает готовое соединение.
|
||||
ParallelReplicas(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_);
|
||||
|
||||
/// Принимает пул, из которого нужно будет достать одно или несколько соединений.
|
||||
ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_);
|
||||
/** Принимает пул, из которого нужно будет достать одно или несколько соединений.
|
||||
* Если флаг append_extra_info установлен, к каждому полученному блоку прилагается
|
||||
* дополнительная информация.
|
||||
* Если флаг get_all_replicas установлен, достаются все соединения.
|
||||
*/
|
||||
ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_,
|
||||
bool append_extra_info = false, bool get_all_replicas = false);
|
||||
|
||||
/// Отправить на реплики всё содержимое внешних таблиц.
|
||||
void sendExternalTablesData(std::vector<ExternalTablesData> & data);
|
||||
@ -36,6 +41,9 @@ public:
|
||||
/// Получить пакет от какой-нибудь реплики.
|
||||
Connection::Packet receivePacket();
|
||||
|
||||
/// Получить информацию про последний полученный пакет.
|
||||
BlockExtraInfo getBlockExtraInfo() const;
|
||||
|
||||
/// Разорвать все действующие соединения.
|
||||
void disconnect();
|
||||
|
||||
@ -94,6 +102,11 @@ private:
|
||||
std::vector<ConnectionPool::Entry> pool_entries;
|
||||
ConnectionPool::Entry pool_entry;
|
||||
|
||||
/// Соединение, c которого был получен последний блок.
|
||||
Connection * current_connection;
|
||||
/// Информация про последний полученный блок, если поддерживается.
|
||||
std::unique_ptr<BlockExtraInfo> block_extra_info;
|
||||
|
||||
/// Текущее количество действительных соединений к репликам.
|
||||
size_t active_replica_count;
|
||||
/// Запрос выполняется параллельно на нескольких репликах.
|
||||
|
@ -230,10 +230,16 @@ public:
|
||||
if (size == 0)
|
||||
return res;
|
||||
|
||||
res_->getData().reserve(size);
|
||||
auto & res_data = res_->getData();
|
||||
|
||||
res_data.reserve(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
if (filter[i])
|
||||
res_->getData().push_back(getData()[i]);
|
||||
res_data.push_back(getData()[i]);
|
||||
|
||||
/// Для экономии оперативки в случае слишком сильной фильтрации.
|
||||
if (res_data.size() * 2 < res_data.capacity())
|
||||
res_data = Container_t(res_data.cbegin(), res_data.cend());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
@ -233,7 +233,13 @@ public:
|
||||
|
||||
void getExtremes(Field & min, Field & max) const override
|
||||
{
|
||||
throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
size_t tuple_size = columns.size();
|
||||
|
||||
min = Array(tuple_size);
|
||||
max = Array(tuple_size);
|
||||
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
columns[i]->getExtremes(min.get<Array &>()[i], max.get<Array &>()[i]);
|
||||
}
|
||||
|
||||
|
||||
|
254
dbms/include/DB/Common/CompactArray.h
Normal file
254
dbms/include/DB/Common/CompactArray.h
Normal file
@ -0,0 +1,254 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/IO/ReadBuffer.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/Core/Defines.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Компактный массив для хранения данных, размер L, в битах, которых составляет
|
||||
* меньше одного байта. Вместо того, чтобы хранить каждое значение в отдельный
|
||||
* байт, что приводит к растрате 37.5% пространства для L=5, CompactArray хранит
|
||||
* смежные L-битные значения в массиве байтов, т.е. фактически CompactArray
|
||||
* симулирует массив L-битных значений.
|
||||
*/
|
||||
template <typename BucketIndex, UInt8 content_width, size_t bucket_count>
|
||||
class __attribute__ ((packed)) CompactArray final
|
||||
{
|
||||
public:
|
||||
class Reader;
|
||||
class Locus;
|
||||
|
||||
public:
|
||||
CompactArray() = default;
|
||||
|
||||
UInt8 ALWAYS_INLINE operator[](BucketIndex bucket_index) const
|
||||
{
|
||||
Locus locus(bucket_index);
|
||||
|
||||
if (locus.index_l == locus.index_r)
|
||||
return locus.read(bitset[locus.index_l]);
|
||||
else
|
||||
return locus.read(bitset[locus.index_l], bitset[locus.index_r]);
|
||||
}
|
||||
|
||||
Locus ALWAYS_INLINE operator[](BucketIndex bucket_index)
|
||||
{
|
||||
Locus locus(bucket_index);
|
||||
|
||||
locus.content_l = &bitset[locus.index_l];
|
||||
|
||||
if (locus.index_l == locus.index_r)
|
||||
locus.content_r = locus.content_l;
|
||||
else
|
||||
locus.content_r = &bitset[locus.index_r];
|
||||
|
||||
return locus;
|
||||
}
|
||||
|
||||
void readText(ReadBuffer & in)
|
||||
{
|
||||
for (size_t i = 0; i < BITSET_SIZE; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
assertString(",", in);
|
||||
readIntText(bitset[i], in);
|
||||
}
|
||||
}
|
||||
|
||||
void writeText(WriteBuffer & out) const
|
||||
{
|
||||
for (size_t i = 0; i < BITSET_SIZE; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
writeCString(",", out);
|
||||
writeIntText(bitset[i], out);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
/// число байт в битсете
|
||||
static constexpr size_t BITSET_SIZE = (static_cast<size_t>(bucket_count) * content_width + 7) / 8;
|
||||
UInt8 bitset[BITSET_SIZE] = { 0 };
|
||||
};
|
||||
|
||||
/** Класс для последовательного чтения ячеек из компактного массива на диске.
|
||||
*/
|
||||
template <typename BucketIndex, UInt8 content_width, size_t bucket_count>
|
||||
class CompactArray<BucketIndex, content_width, bucket_count>::Reader final
|
||||
{
|
||||
public:
|
||||
Reader(ReadBuffer & in_)
|
||||
: in(in_)
|
||||
{
|
||||
}
|
||||
|
||||
Reader(const Reader &) = delete;
|
||||
Reader & operator=(const Reader &) = delete;
|
||||
|
||||
bool next()
|
||||
{
|
||||
if (current_bucket_index == bucket_count)
|
||||
{
|
||||
is_eof = true;
|
||||
return false;
|
||||
}
|
||||
|
||||
locus.init(current_bucket_index);
|
||||
|
||||
if (current_bucket_index == 0)
|
||||
{
|
||||
in.readStrict(reinterpret_cast<char *>(&value_l), 1);
|
||||
++read_count;
|
||||
}
|
||||
else
|
||||
value_l = value_r;
|
||||
|
||||
if (locus.index_l != locus.index_r)
|
||||
{
|
||||
if (read_count == BITSET_SIZE)
|
||||
fits_in_byte = true;
|
||||
else
|
||||
{
|
||||
fits_in_byte = false;
|
||||
in.readStrict(reinterpret_cast<char *>(&value_r), 1);
|
||||
++read_count;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
fits_in_byte = true;
|
||||
value_r = value_l;
|
||||
}
|
||||
|
||||
++current_bucket_index;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Вернуть текущий номер ячейки и соответствующее содержание.
|
||||
*/
|
||||
inline std::pair<BucketIndex, UInt8> get() const
|
||||
{
|
||||
if ((current_bucket_index == 0) || is_eof)
|
||||
throw Exception("No available data.", ErrorCodes::NO_AVAILABLE_DATA);
|
||||
|
||||
if (fits_in_byte)
|
||||
return std::make_pair(current_bucket_index - 1, locus.read(value_l));
|
||||
else
|
||||
return std::make_pair(current_bucket_index - 1, locus.read(value_l, value_r));
|
||||
}
|
||||
|
||||
private:
|
||||
ReadBuffer & in;
|
||||
/// Физическое расположение текущей ячейки.
|
||||
Locus locus;
|
||||
/// Текущая позиция в файле в виде номера ячейки.
|
||||
BucketIndex current_bucket_index = 0;
|
||||
/// Количество прочитанных байтов.
|
||||
size_t read_count = 0;
|
||||
/// Содержание в текущей позиции.
|
||||
UInt8 value_l;
|
||||
UInt8 value_r;
|
||||
///
|
||||
bool is_eof = false;
|
||||
/// Влезает ли ячейка полностью в один байт?
|
||||
bool fits_in_byte;
|
||||
};
|
||||
|
||||
/** Структура Locus содержит необходимую информацию, чтобы найти для каждой ячейки
|
||||
* соответствующие байт и смещение, в битах, от начала ячейки. Поскольку в общем
|
||||
* случае размер одного байта не делится на размер одной ячейки, возможны случаи,
|
||||
* когда одна ячейка перекрывает два байта. Поэтому структура Locus содержит две
|
||||
* пары (индекс, смещение).
|
||||
*/
|
||||
template <typename BucketIndex, UInt8 content_width, size_t bucket_count>
|
||||
class CompactArray<BucketIndex, content_width, bucket_count>::Locus final
|
||||
{
|
||||
friend class CompactArray;
|
||||
friend class CompactArray::Reader;
|
||||
|
||||
public:
|
||||
ALWAYS_INLINE operator UInt8() const
|
||||
{
|
||||
if (content_l == content_r)
|
||||
return read(*content_l);
|
||||
else
|
||||
return read(*content_l, *content_r);
|
||||
}
|
||||
|
||||
Locus ALWAYS_INLINE & operator=(UInt8 content)
|
||||
{
|
||||
if ((index_l == index_r) || (index_l == (BITSET_SIZE - 1)))
|
||||
{
|
||||
/// Ячейка полностью влезает в один байт.
|
||||
*content_l &= ~(((1 << content_width) - 1) << offset_l);
|
||||
*content_l |= content << offset_l;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Ячейка перекрывает два байта.
|
||||
size_t left = 8 - offset_l;
|
||||
|
||||
*content_l &= ~(((1 << left) - 1) << offset_l);
|
||||
*content_l |= (content & ((1 << left) - 1)) << offset_l;
|
||||
|
||||
*content_r &= ~((1 << offset_r) - 1);
|
||||
*content_r |= content >> left;
|
||||
}
|
||||
|
||||
return *this;
|
||||
}
|
||||
|
||||
private:
|
||||
Locus() = default;
|
||||
|
||||
Locus(BucketIndex bucket_index)
|
||||
{
|
||||
init(bucket_index);
|
||||
}
|
||||
|
||||
void ALWAYS_INLINE init(BucketIndex bucket_index)
|
||||
{
|
||||
size_t l = static_cast<size_t>(bucket_index) * content_width;
|
||||
index_l = l >> 3;
|
||||
offset_l = l & 7;
|
||||
|
||||
size_t r = static_cast<size_t>(bucket_index + 1) * content_width;
|
||||
index_r = r >> 3;
|
||||
offset_r = r & 7;
|
||||
}
|
||||
|
||||
UInt8 ALWAYS_INLINE read(UInt8 value_l) const
|
||||
{
|
||||
/// Ячейка полностью влезает в один байт.
|
||||
return (value_l >> offset_l) & ((1 << content_width) - 1);
|
||||
}
|
||||
|
||||
UInt8 ALWAYS_INLINE read(UInt8 value_l, UInt8 value_r) const
|
||||
{
|
||||
/// Ячейка перекрывает два байта.
|
||||
return ((value_l >> offset_l) & ((1 << (8 - offset_l)) - 1))
|
||||
| ((value_r & ((1 << offset_r) - 1)) << (8 - offset_l));
|
||||
}
|
||||
|
||||
private:
|
||||
size_t index_l;
|
||||
size_t offset_l;
|
||||
size_t index_r;
|
||||
size_t offset_r;
|
||||
|
||||
UInt8 * content_l;
|
||||
UInt8 * content_r;
|
||||
|
||||
/// Проверки
|
||||
static_assert((content_width > 0) && (content_width < 8), "Invalid parameter value");
|
||||
static_assert(bucket_count <= (std::numeric_limits<size_t>::max() / content_width), "Invalid parameter value");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <common/Common.h>
|
||||
#include <stats/IntHash.h>
|
||||
#include <DB/Common/HyperLogLogBiasEstimator.h>
|
||||
#include <DB/Common/CompactArray.h>
|
||||
|
||||
#include <DB/IO/ReadBuffer.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
@ -61,154 +62,6 @@ template<UInt64 MaxValue> struct MinCounterType
|
||||
>::Type Type;
|
||||
};
|
||||
|
||||
/** Компактный массив для хранения данных, размер L, в битах, которых составляет меньше одного байта.
|
||||
* Вместо того, чтобы хранить каждое значение в 8-битную ячейку памяти, что приводит к растрате
|
||||
* 37.5% пространства для L=5, CompactArray хранит смежные L-битные значения, именно компактные
|
||||
* ячейки в массиве байтов, т.е. фактически CompactArray симулирует массив L-битных значений.
|
||||
*/
|
||||
template<typename BucketIndex, UInt8 content_width, size_t bucket_count>
|
||||
class __attribute__ ((packed)) CompactArray final
|
||||
{
|
||||
public:
|
||||
class Locus;
|
||||
|
||||
public:
|
||||
CompactArray() = default;
|
||||
|
||||
UInt8 ALWAYS_INLINE operator[](BucketIndex bucket_index) const
|
||||
{
|
||||
Locus locus(bucket_index);
|
||||
|
||||
if (locus.index_l == locus.index_r)
|
||||
return locus.read(bitset[locus.index_l]);
|
||||
else
|
||||
return locus.read(bitset[locus.index_l], bitset[locus.index_r]);
|
||||
}
|
||||
|
||||
Locus ALWAYS_INLINE operator[](BucketIndex bucket_index)
|
||||
{
|
||||
Locus locus(bucket_index);
|
||||
|
||||
locus.content_l = &bitset[locus.index_l];
|
||||
|
||||
if (locus.index_l == locus.index_r)
|
||||
locus.content_r = locus.content_l;
|
||||
else
|
||||
locus.content_r = &bitset[locus.index_r];
|
||||
|
||||
return locus;
|
||||
}
|
||||
|
||||
void readText(DB::ReadBuffer & in)
|
||||
{
|
||||
for (size_t i = 0; i < BITSET_SIZE; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
DB::assertString(",", in);
|
||||
DB::readIntText(bitset[i], in);
|
||||
}
|
||||
}
|
||||
|
||||
void writeText(DB::WriteBuffer & out) const
|
||||
{
|
||||
for (size_t i = 0; i < BITSET_SIZE; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
writeCString(",", out);
|
||||
DB::writeIntText(bitset[i], out);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
/// число байт в битсете
|
||||
static constexpr size_t BITSET_SIZE = (static_cast<size_t>(bucket_count) * content_width + 7) / 8;
|
||||
UInt8 bitset[BITSET_SIZE] = { 0 };
|
||||
};
|
||||
|
||||
/** Структура Locus содержит необходимую информацию, чтобы найти для каждой компактной ячейки
|
||||
* соответствующие физическую ячейку и смещение, в битах, от начала ячейки. Поскольку в общем
|
||||
* случае размер одной физической ячейки не делится на размер одной компактной ячейки, возможны
|
||||
* случаи, когда одна компактная ячейка перекрывает две физические ячейки. Поэтому структура
|
||||
* Locus содержит две пары (индекс, смещение).
|
||||
*/
|
||||
template<typename BucketIndex, UInt8 content_width, size_t bucket_count>
|
||||
class CompactArray<BucketIndex, content_width, bucket_count>::Locus final
|
||||
{
|
||||
friend class CompactArray;
|
||||
|
||||
public:
|
||||
ALWAYS_INLINE operator UInt8() const
|
||||
{
|
||||
if (content_l == content_r)
|
||||
return read(*content_l);
|
||||
else
|
||||
return read(*content_l, *content_r);
|
||||
}
|
||||
|
||||
Locus ALWAYS_INLINE & operator=(UInt8 content)
|
||||
{
|
||||
if ((index_l == index_r) || (index_l == (BITSET_SIZE - 1)))
|
||||
{
|
||||
/// Компактная ячейка полностью влезает в одну физическую ячейку.
|
||||
*content_l &= ~(((1 << content_width) - 1) << offset_l);
|
||||
*content_l |= content << offset_l;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Компактная ячейка перекрывает две физические ячейки.
|
||||
size_t left = 8 - offset_l;
|
||||
|
||||
*content_l &= ~(((1 << left) - 1) << offset_l);
|
||||
*content_l |= (content & ((1 << left) - 1)) << offset_l;
|
||||
|
||||
*content_r &= ~((1 << offset_r) - 1);
|
||||
*content_r |= content >> left;
|
||||
}
|
||||
|
||||
return *this;
|
||||
}
|
||||
|
||||
private:
|
||||
Locus() = default;
|
||||
|
||||
Locus(BucketIndex bucket_index)
|
||||
{
|
||||
size_t l = static_cast<size_t>(bucket_index) * content_width;
|
||||
index_l = l >> 3;
|
||||
offset_l = l & 7;
|
||||
|
||||
size_t r = static_cast<size_t>(bucket_index + 1) * content_width;
|
||||
index_r = r >> 3;
|
||||
offset_r = r & 7;
|
||||
}
|
||||
|
||||
UInt8 ALWAYS_INLINE read(UInt8 value_l) const
|
||||
{
|
||||
/// Компактная ячейка полностью влезает в одну физическую ячейку.
|
||||
return (value_l >> offset_l) & ((1 << content_width) - 1);
|
||||
}
|
||||
|
||||
UInt8 ALWAYS_INLINE read(UInt8 value_l, UInt8 value_r) const
|
||||
{
|
||||
/// Компактная ячейка перекрывает две физические ячейки.
|
||||
return ((value_l >> offset_l) & ((1 << (8 - offset_l)) - 1))
|
||||
| ((value_r & ((1 << offset_r) - 1)) << (8 - offset_l));
|
||||
}
|
||||
|
||||
private:
|
||||
size_t index_l;
|
||||
size_t offset_l;
|
||||
size_t index_r;
|
||||
size_t offset_r;
|
||||
|
||||
UInt8 * content_l;
|
||||
UInt8 * content_r;
|
||||
|
||||
/// Проверки
|
||||
static_assert((content_width > 0) && (content_width < 8), "Invalid parameter value");
|
||||
static_assert(bucket_count <= (std::numeric_limits<size_t>::max() / content_width), "Invalid parameter value");
|
||||
};
|
||||
|
||||
/** Знаменатель формулы алгоритма HyperLogLog
|
||||
*/
|
||||
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
|
||||
@ -422,7 +275,7 @@ private:
|
||||
|
||||
private:
|
||||
using Value_t = UInt64;
|
||||
using RankStore = details::CompactArray<HashValueType, rank_width, bucket_count>;
|
||||
using RankStore = DB::CompactArray<HashValueType, rank_width, bucket_count>;
|
||||
|
||||
public:
|
||||
void insert(Value_t value)
|
||||
@ -476,12 +329,11 @@ public:
|
||||
|
||||
void readAndMerge(DB::ReadBuffer & in)
|
||||
{
|
||||
RankStore other;
|
||||
in.readStrict(reinterpret_cast<char *>(&other), sizeof(RankStore));
|
||||
for (HashValueType bucket = 0; bucket < bucket_count; ++bucket)
|
||||
typename RankStore::Reader reader(in);
|
||||
while (reader.next())
|
||||
{
|
||||
UInt8 rank = other[bucket];
|
||||
update(bucket, rank);
|
||||
const auto & data = reader.get();
|
||||
update(data.first, data.second);
|
||||
}
|
||||
|
||||
in.ignore(sizeof(DenominatorCalculatorType) + sizeof(ZerosCounterType));
|
||||
|
@ -67,7 +67,7 @@ private:
|
||||
return n;
|
||||
}
|
||||
|
||||
static size_t to_size(size_t n) { return byte_size(std::max(POD_ARRAY_INITIAL_SIZE, round_up_to_power_of_two(n))); }
|
||||
static size_t to_size(size_t n) { return byte_size(round_up_to_power_of_two(n)); }
|
||||
|
||||
void alloc(size_t n)
|
||||
{
|
||||
@ -162,6 +162,8 @@ public:
|
||||
iterator end() { return t_end(); }
|
||||
const_iterator begin() const { return t_start(); }
|
||||
const_iterator end() const { return t_end(); }
|
||||
const_iterator cbegin() const { return t_start(); }
|
||||
const_iterator cend() const { return t_end(); }
|
||||
|
||||
void reserve(size_t n)
|
||||
{
|
||||
|
@ -103,11 +103,16 @@ public:
|
||||
/** Выделяет до указанного количества соединений для работы
|
||||
* Соединения предоставляют доступ к разным репликам одного шарда.
|
||||
*/
|
||||
std::vector<Entry> getMany(const DB::Settings * settings)
|
||||
std::vector<Entry> getMany(const DB::Settings * settings, bool get_all)
|
||||
{
|
||||
ResourceTracker resource_tracker{nested_pools.size()};
|
||||
|
||||
UInt64 max_connections = settings ? UInt64(settings->max_parallel_replicas) : 1;
|
||||
UInt64 max_connections;
|
||||
if (get_all)
|
||||
max_connections = nested_pools.size();
|
||||
else
|
||||
max_connections = settings ? UInt64(settings->max_parallel_replicas) : 1;
|
||||
|
||||
bool skip_unavailable = settings ? UInt64(settings->skip_unavailable_shards) : false;
|
||||
|
||||
std::vector<Entry> connections;
|
||||
@ -120,7 +125,7 @@ public:
|
||||
|
||||
if (getResource(entry, fail_messages, &resource_tracker, settings))
|
||||
connections.push_back(entry);
|
||||
else if (i == 0 && !skip_unavailable)
|
||||
else if (get_all || ((i == 0) && !skip_unavailable))
|
||||
throw DB::NetException("All connection tries failed. Log: \n\n" + fail_messages.str() + "\n", DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED);
|
||||
else
|
||||
break;
|
||||
@ -129,7 +134,6 @@ public:
|
||||
return connections;
|
||||
}
|
||||
|
||||
|
||||
protected:
|
||||
struct PoolWithErrorCount
|
||||
{
|
||||
|
639
dbms/include/DB/Common/StringSearcher.h
Normal file
639
dbms/include/DB/Common/StringSearcher.h
Normal file
@ -0,0 +1,639 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Common/UTF8Helpers.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
#include <Poco/Unicode.h>
|
||||
#include <stdint.h>
|
||||
#include <string.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/// Performs case-sensitive and case-insensitive search of UTF-8 strings
|
||||
template <bool CaseSensitive, bool ASCII> class StringSearcher;
|
||||
|
||||
/// Case-insensitive UTF-8 searcher
|
||||
template <> class StringSearcher<false, false>
|
||||
{
|
||||
using UTF8SequenceBuffer = UInt8[6];
|
||||
|
||||
static constexpr auto n = sizeof(__m128i);
|
||||
|
||||
const int page_size = getpagesize();
|
||||
|
||||
/// string to be searched for
|
||||
const UInt8 * const needle;
|
||||
const std::size_t needle_size;
|
||||
const UInt8 * const needle_end = needle + needle_size;
|
||||
/// lower and uppercase variants of the first octet of the first character in `needle`
|
||||
bool first_needle_symbol_is_ascii{};
|
||||
UInt8 l{};
|
||||
UInt8 u{};
|
||||
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
|
||||
__m128i patl, patu;
|
||||
/// lower and uppercase vectors of first 16 characters of `needle`
|
||||
__m128i cachel = _mm_setzero_si128(), cacheu = _mm_setzero_si128();
|
||||
int cachemask{};
|
||||
std::size_t cache_valid_len{};
|
||||
std::size_t cache_actual_len{};
|
||||
|
||||
bool page_safe(const void * const ptr) const
|
||||
{
|
||||
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
|
||||
}
|
||||
|
||||
public:
|
||||
StringSearcher(const char * const needle_, const std::size_t needle_size)
|
||||
: needle{reinterpret_cast<const UInt8 *>(needle_)}, needle_size{needle_size}
|
||||
{
|
||||
if (0 == needle_size)
|
||||
return;
|
||||
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
UTF8SequenceBuffer l_seq, u_seq;
|
||||
|
||||
if (*needle < 0x80u)
|
||||
{
|
||||
first_needle_symbol_is_ascii = true;
|
||||
l = static_cast<const UInt8>(std::tolower(*needle));
|
||||
u = static_cast<const UInt8>(std::toupper(*needle));
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto first_u32 = utf8.convert(needle);
|
||||
const auto first_l_u32 = Poco::Unicode::toLower(first_u32);
|
||||
const auto first_u_u32 = Poco::Unicode::toUpper(first_u32);
|
||||
|
||||
/// lower and uppercase variants of the first octet of the first character in `needle`
|
||||
utf8.convert(first_l_u32, l_seq, sizeof(l_seq));
|
||||
l = l_seq[0];
|
||||
utf8.convert(first_u_u32, u_seq, sizeof(u_seq));
|
||||
u = u_seq[0];
|
||||
}
|
||||
|
||||
/// for detecting leftmost position of the first symbol
|
||||
patl = _mm_set1_epi8(l);
|
||||
patu = _mm_set1_epi8(u);
|
||||
/// lower and uppercase vectors of first 16 octets of `needle`
|
||||
|
||||
auto needle_pos = needle;
|
||||
|
||||
for (std::size_t i = 0; i < n;)
|
||||
{
|
||||
if (needle_pos == needle_end)
|
||||
{
|
||||
cachel = _mm_srli_si128(cachel, 1);
|
||||
cacheu = _mm_srli_si128(cacheu, 1);
|
||||
++i;
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto src_len = DB::UTF8::seqLength(*needle_pos);
|
||||
const auto c_u32 = utf8.convert(needle_pos);
|
||||
|
||||
const auto c_l_u32 = Poco::Unicode::toLower(c_u32);
|
||||
const auto c_u_u32 = Poco::Unicode::toUpper(c_u32);
|
||||
|
||||
const auto dst_l_len = static_cast<UInt8>(utf8.convert(c_l_u32, l_seq, sizeof(l_seq)));
|
||||
const auto dst_u_len = static_cast<UInt8>(utf8.convert(c_u_u32, u_seq, sizeof(u_seq)));
|
||||
|
||||
/// @note Unicode standard states it is a rare but possible occasion
|
||||
if (!(dst_l_len == dst_u_len && dst_u_len == src_len))
|
||||
throw DB::Exception{
|
||||
"UTF8 sequences with different lowercase and uppercase lengths are not supported",
|
||||
DB::ErrorCodes::UNSUPPORTED_PARAMETER
|
||||
};
|
||||
|
||||
cache_actual_len += src_len;
|
||||
if (cache_actual_len < n)
|
||||
cache_valid_len += src_len;
|
||||
|
||||
for (std::size_t j = 0; j < src_len && i < n; ++j, ++i)
|
||||
{
|
||||
cachel = _mm_srli_si128(cachel, 1);
|
||||
cacheu = _mm_srli_si128(cacheu, 1);
|
||||
|
||||
if (needle_pos != needle_end)
|
||||
{
|
||||
cachel = _mm_insert_epi8(cachel, l_seq[j], n - 1);
|
||||
cacheu = _mm_insert_epi8(cacheu, u_seq[j], n - 1);
|
||||
|
||||
cachemask |= 1 << i;
|
||||
++needle_pos;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool compare(const UInt8 * pos) const
|
||||
{
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
|
||||
if (page_safe(pos))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (0xffff == cachemask)
|
||||
{
|
||||
if (mask == cachemask)
|
||||
{
|
||||
pos += cache_valid_len;
|
||||
auto needle_pos = needle + cache_valid_len;
|
||||
|
||||
while (needle_pos < needle_end &&
|
||||
Poco::Unicode::toLower(utf8.convert(pos)) ==
|
||||
Poco::Unicode::toLower(utf8.convert(needle_pos)))
|
||||
{
|
||||
/// @note assuming sequences for lowercase and uppercase have exact same length
|
||||
const auto len = DB::UTF8::seqLength(*pos);
|
||||
pos += len, needle_pos += len;
|
||||
}
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
else if ((mask & cachemask) == cachemask)
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
if (*pos == l || *pos == u)
|
||||
{
|
||||
pos += first_needle_symbol_is_ascii;
|
||||
auto needle_pos = needle + first_needle_symbol_is_ascii;
|
||||
|
||||
while (needle_pos < needle_end &&
|
||||
Poco::Unicode::toLower(utf8.convert(pos)) ==
|
||||
Poco::Unicode::toLower(utf8.convert(needle_pos)))
|
||||
{
|
||||
const auto len = DB::UTF8::seqLength(*pos);
|
||||
pos += len, needle_pos += len;
|
||||
}
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
const UInt8 * search(const UInt8 * haystack, const UInt8 * const haystack_end) const
|
||||
{
|
||||
if (0 == needle_size)
|
||||
return haystack;
|
||||
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
|
||||
while (haystack < haystack_end)
|
||||
{
|
||||
if (haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, patl);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, patu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (mask == 0)
|
||||
{
|
||||
haystack += n;
|
||||
DB::UTF8::syncForward(haystack, haystack_end);
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto offset = _bit_scan_forward(mask);
|
||||
haystack += offset;
|
||||
|
||||
if (haystack < haystack_end && haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (0xffff == cachemask)
|
||||
{
|
||||
if (mask == cachemask)
|
||||
{
|
||||
auto haystack_pos = haystack + cache_valid_len;
|
||||
auto needle_pos = needle + cache_valid_len;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
Poco::Unicode::toLower(utf8.convert(haystack_pos)) ==
|
||||
Poco::Unicode::toLower(utf8.convert(needle_pos)))
|
||||
{
|
||||
/// @note assuming sequences for lowercase and uppercase have exact same length
|
||||
const auto len = DB::UTF8::seqLength(*haystack_pos);
|
||||
haystack_pos += len, needle_pos += len;
|
||||
}
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
}
|
||||
else if ((mask & cachemask) == cachemask)
|
||||
return haystack;
|
||||
|
||||
/// first octet was ok, but not the first 16, move to start of next sequence and reapply
|
||||
haystack += DB::UTF8::seqLength(*haystack);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
if (haystack == haystack_end)
|
||||
return haystack_end;
|
||||
|
||||
if (*haystack == l || *haystack == u)
|
||||
{
|
||||
auto haystack_pos = haystack + first_needle_symbol_is_ascii;
|
||||
auto needle_pos = needle + first_needle_symbol_is_ascii;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
Poco::Unicode::toLower(utf8.convert(haystack_pos)) ==
|
||||
Poco::Unicode::toLower(utf8.convert(needle_pos)))
|
||||
{
|
||||
const auto len = DB::UTF8::seqLength(*haystack_pos);
|
||||
haystack_pos += len, needle_pos += len;
|
||||
}
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
|
||||
/// advance to the start of the next sequence
|
||||
haystack += DB::UTF8::seqLength(*haystack);
|
||||
}
|
||||
|
||||
return haystack_end;
|
||||
}
|
||||
};
|
||||
|
||||
/// Case-insensitive ASCII searcher
|
||||
template <> class StringSearcher<false, true>
|
||||
{
|
||||
static constexpr auto n = sizeof(__m128i);
|
||||
|
||||
const int page_size = getpagesize();
|
||||
|
||||
/// string to be searched for
|
||||
const UInt8 * const needle;
|
||||
const std::size_t needle_size;
|
||||
const UInt8 * const needle_end = needle + needle_size;
|
||||
/// lower and uppercase variants of the first character in `needle`
|
||||
UInt8 l{};
|
||||
UInt8 u{};
|
||||
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
|
||||
__m128i patl, patu;
|
||||
/// lower and uppercase vectors of first 16 characters of `needle`
|
||||
__m128i cachel = _mm_setzero_si128(), cacheu = _mm_setzero_si128();
|
||||
int cachemask{};
|
||||
|
||||
bool page_safe(const void * const ptr) const
|
||||
{
|
||||
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
|
||||
}
|
||||
|
||||
public:
|
||||
StringSearcher(const char * const needle_, const std::size_t needle_size)
|
||||
: needle{reinterpret_cast<const UInt8 *>(needle_)}, needle_size{needle_size}
|
||||
{
|
||||
if (0 == needle_size)
|
||||
return;
|
||||
|
||||
l = static_cast<UInt8>(std::tolower(*needle));
|
||||
u = static_cast<UInt8>(std::toupper(*needle));
|
||||
|
||||
patl = _mm_set1_epi8(l);
|
||||
patu = _mm_set1_epi8(u);
|
||||
|
||||
auto needle_pos = needle;
|
||||
|
||||
for (const auto i : ext::range(0, n))
|
||||
{
|
||||
cachel = _mm_srli_si128(cachel, 1);
|
||||
cacheu = _mm_srli_si128(cacheu, 1);
|
||||
|
||||
if (needle_pos != needle_end)
|
||||
{
|
||||
cachel = _mm_insert_epi8(cachel, std::tolower(*needle_pos), n - 1);
|
||||
cacheu = _mm_insert_epi8(cacheu, std::toupper(*needle_pos), n - 1);
|
||||
cachemask |= 1 << i;
|
||||
++needle_pos;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool compare(const UInt8 * pos) const
|
||||
{
|
||||
if (page_safe(pos))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (0xffff == cachemask)
|
||||
{
|
||||
if (mask == cachemask)
|
||||
{
|
||||
pos += n;
|
||||
auto needle_pos = needle + n;
|
||||
|
||||
while (needle_pos < needle_end && std::tolower(*pos) == std::tolower(*needle_pos))
|
||||
++pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
else if ((mask & cachemask) == cachemask)
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
if (*pos == l || *pos == u)
|
||||
{
|
||||
++pos;
|
||||
auto needle_pos = needle + 1;
|
||||
|
||||
while (needle_pos < needle_end && std::tolower(*pos) == std::tolower(*needle_pos))
|
||||
++pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
const UInt8 * search(const UInt8 * haystack, const UInt8 * const haystack_end) const
|
||||
{
|
||||
if (0 == needle_size)
|
||||
return haystack;
|
||||
|
||||
while (haystack < haystack_end)
|
||||
{
|
||||
if (haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, patl);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, patu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (mask == 0)
|
||||
{
|
||||
haystack += n;
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto offset = _bit_scan_forward(mask);
|
||||
haystack += offset;
|
||||
|
||||
if (haystack < haystack_end && haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (0xffff == cachemask)
|
||||
{
|
||||
if (mask == cachemask)
|
||||
{
|
||||
auto haystack_pos = haystack + n;
|
||||
auto needle_pos = needle + n;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
|
||||
++haystack_pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
}
|
||||
else if ((mask & cachemask) == cachemask)
|
||||
return haystack;
|
||||
|
||||
++haystack;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
if (haystack == haystack_end)
|
||||
return haystack_end;
|
||||
|
||||
if (*haystack == l || *haystack == u)
|
||||
{
|
||||
auto haystack_pos = haystack + 1;
|
||||
auto needle_pos = needle + 1;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
|
||||
++haystack_pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
|
||||
++haystack;
|
||||
}
|
||||
|
||||
return haystack_end;
|
||||
}
|
||||
};
|
||||
|
||||
/// Case-sensitive searcher (both ASCII and UTF-8)
|
||||
template <bool ASCII> class StringSearcher<true, ASCII>
|
||||
{
|
||||
static constexpr auto n = sizeof(__m128i);
|
||||
|
||||
const int page_size = getpagesize();
|
||||
|
||||
/// string to be searched for
|
||||
const UInt8 * const needle;
|
||||
const std::size_t needle_size;
|
||||
const UInt8 * const needle_end = needle + needle_size;
|
||||
/// first character in `needle`
|
||||
UInt8 first{};
|
||||
/// vector filled `first` for determining leftmost position of the first symbol
|
||||
__m128i pattern;
|
||||
/// vector of first 16 characters of `needle`
|
||||
__m128i cache = _mm_setzero_si128();
|
||||
int cachemask{};
|
||||
|
||||
bool page_safe(const void * const ptr) const
|
||||
{
|
||||
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
|
||||
}
|
||||
|
||||
public:
|
||||
StringSearcher(const char * const needle_, const std::size_t needle_size)
|
||||
: needle{reinterpret_cast<const UInt8 *>(needle_)}, needle_size{needle_size}
|
||||
{
|
||||
if (0 == needle_size)
|
||||
return;
|
||||
|
||||
first = *needle;
|
||||
pattern = _mm_set1_epi8(first);
|
||||
|
||||
auto needle_pos = needle;
|
||||
|
||||
for (const auto i : ext::range(0, n))
|
||||
{
|
||||
cache = _mm_srli_si128(cache, 1);
|
||||
|
||||
if (needle_pos != needle_end)
|
||||
{
|
||||
cache = _mm_insert_epi8(cache, *needle_pos, n - 1);
|
||||
cachemask |= 1 << i;
|
||||
++needle_pos;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool compare(const UInt8 * pos) const
|
||||
{
|
||||
if (page_safe(pos))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
|
||||
const auto v_against_cache = _mm_cmpeq_epi8(v_haystack, cache);
|
||||
const auto mask = _mm_movemask_epi8(v_against_cache);
|
||||
|
||||
if (0xffff == cachemask)
|
||||
{
|
||||
if (mask == cachemask)
|
||||
{
|
||||
pos += n;
|
||||
auto needle_pos = needle + n;
|
||||
|
||||
while (needle_pos < needle_end && *pos == *needle_pos)
|
||||
++pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
else if ((mask & cachemask) == cachemask)
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
if (*pos == first)
|
||||
{
|
||||
++pos;
|
||||
auto needle_pos = needle + 1;
|
||||
|
||||
while (needle_pos < needle_end && *pos == *needle_pos)
|
||||
++pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
const UInt8 * search(const UInt8 * haystack, const UInt8 * const haystack_end) const
|
||||
{
|
||||
if (0 == needle_size)
|
||||
return haystack;
|
||||
|
||||
while (haystack < haystack_end)
|
||||
{
|
||||
if (haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
/// find first character
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_pattern = _mm_cmpeq_epi8(v_haystack, pattern);
|
||||
|
||||
const auto mask = _mm_movemask_epi8(v_against_pattern);
|
||||
|
||||
/// first character not present in 16 octets starting at `haystack`
|
||||
if (mask == 0)
|
||||
{
|
||||
haystack += n;
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto offset = _bit_scan_forward(mask);
|
||||
haystack += offset;
|
||||
|
||||
if (haystack < haystack_end && haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
/// check for first 16 octets
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_cache = _mm_cmpeq_epi8(v_haystack, cache);
|
||||
const auto mask = _mm_movemask_epi8(v_against_cache);
|
||||
|
||||
if (0xffff == cachemask)
|
||||
{
|
||||
if (mask == cachemask)
|
||||
{
|
||||
auto haystack_pos = haystack + n;
|
||||
auto needle_pos = needle + n;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
*haystack_pos == *needle_pos)
|
||||
++haystack_pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
}
|
||||
else if ((mask & cachemask) == cachemask)
|
||||
return haystack;
|
||||
|
||||
++haystack;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
if (haystack == haystack_end)
|
||||
return haystack_end;
|
||||
|
||||
if (*haystack == first)
|
||||
{
|
||||
auto haystack_pos = haystack + 1;
|
||||
auto needle_pos = needle + 1;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
*haystack_pos == *needle_pos)
|
||||
++haystack_pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
|
||||
++haystack;
|
||||
}
|
||||
|
||||
return haystack_end;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
using ASCIICaseSensitiveStringSearcher = StringSearcher<true, true>;
|
||||
using ASCIICaseInsensitiveStringSearcher = StringSearcher<false, true>;
|
||||
using UTF8CaseSensitiveStringSearcher = StringSearcher<true, false>;
|
||||
using UTF8CaseInsensitiveStringSearcher = StringSearcher<false, false>;
|
||||
|
||||
|
||||
}
|
54
dbms/include/DB/Common/UTF8Helpers.h
Normal file
54
dbms/include/DB/Common/UTF8Helpers.h
Normal file
@ -0,0 +1,54 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/Types.h>
|
||||
#include <x86intrin.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
namespace UTF8
|
||||
{
|
||||
|
||||
|
||||
static const UInt8 CONTINUATION_OCTET_MASK = 0b11000000u;
|
||||
static const UInt8 CONTINUATION_OCTET = 0b10000000u;
|
||||
|
||||
/// return true if `octet` binary repr starts with 10 (octet is a UTF-8 sequence continuation)
|
||||
inline bool isContinuationOctet(const UInt8 octet)
|
||||
{
|
||||
return (octet & CONTINUATION_OCTET_MASK) == CONTINUATION_OCTET;
|
||||
}
|
||||
|
||||
/// moves `s` backward until either first non-continuation octet
|
||||
inline void syncBackward(const UInt8 * & s)
|
||||
{
|
||||
while (isContinuationOctet(*s))
|
||||
--s;
|
||||
}
|
||||
|
||||
/// moves `s` forward until either first non-continuation octet or string end is met
|
||||
inline void syncForward(const UInt8 * & s, const UInt8 * const end)
|
||||
{
|
||||
while (s < end && isContinuationOctet(*s))
|
||||
++s;
|
||||
}
|
||||
|
||||
/// returns UTF-8 code point sequence length judging by it's first octet
|
||||
inline std::size_t seqLength(const UInt8 first_octet)
|
||||
{
|
||||
if (first_octet < 0x80u)
|
||||
return 1;
|
||||
|
||||
const std::size_t bits = 8;
|
||||
const auto first_zero = _bit_scan_reverse(static_cast<UInt8>(~first_octet));
|
||||
|
||||
return bits - 1 - first_zero;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -1,5 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Common/StringSearcher.h>
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
#include <Poco/Unicode.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <x86intrin.h>
|
||||
#include <stdint.h>
|
||||
#include <string.h>
|
||||
|
||||
@ -23,111 +28,426 @@
|
||||
*
|
||||
* Используется невыровненный доступ к памяти.
|
||||
*/
|
||||
class Volnitsky
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
private:
|
||||
typedef uint8_t offset_t; /// Смещение в needle. Для основного алгоритма, длина needle не должна быть больше 255.
|
||||
typedef uint16_t ngram_t; /// n-грамма (2 байта).
|
||||
|
||||
const char * needle;
|
||||
size_t needle_size;
|
||||
const char * needle_end;
|
||||
size_t step; /// Насколько двигаемся, если n-грамма из haystack не нашлась в хэш-таблице.
|
||||
|
||||
/// @todo store lowercase needle to speed up in case there are numerous occurrences of bigrams from needle in haystack
|
||||
template <typename CRTP>
|
||||
class VolnitskyBase
|
||||
{
|
||||
protected:
|
||||
using offset_t = uint8_t; /// Смещение в needle. Для основного алгоритма, длина needle не должна быть больше 255.
|
||||
using ngram_t = uint16_t; /// n-грамма (2 байта).
|
||||
|
||||
const UInt8 * const needle;
|
||||
const size_t needle_size;
|
||||
const UInt8 * const needle_end = needle + needle_size;
|
||||
/// На сколько двигаемся, если n-грамма из haystack не нашлась в хэш-таблице.
|
||||
const size_t step = needle_size - sizeof(ngram_t) + 1;
|
||||
|
||||
/** max needle length is 255, max distinct ngrams for case-sensitive is (255 - 1), case-insensitive is 4 * (255 - 1)
|
||||
* storage of 64K ngrams (n = 2, 128 KB) should be large enough for both cases */
|
||||
static const size_t hash_size = 64 * 1024; /// Помещается в L2-кэш.
|
||||
offset_t hash[hash_size]; /// Хэш-таблица.
|
||||
|
||||
bool fallback; /// Нужно ли использовать fallback алгоритм.
|
||||
|
||||
/// fallback алгоритм
|
||||
static const char * naive_memmem(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size)
|
||||
{
|
||||
const char * pos = haystack;
|
||||
const char * end = haystack + haystack_size;
|
||||
while (nullptr != (pos = reinterpret_cast<const char *>(memchr(pos, needle[0], end - pos))) && pos + needle_size <= end)
|
||||
{
|
||||
if (0 == memcmp(pos, needle, needle_size))
|
||||
return pos;
|
||||
else
|
||||
++pos;
|
||||
}
|
||||
|
||||
return end;
|
||||
}
|
||||
/// min haystack size to use main algorithm instead of fallback
|
||||
static constexpr auto min_haystack_size_for_algorithm = 20000;
|
||||
const bool fallback; /// Нужно ли использовать fallback алгоритм.
|
||||
|
||||
public:
|
||||
/** haystack_size_hint - ожидаемый суммарный размер haystack при вызовах search. Можно не указывать.
|
||||
* Если указать его достаточно маленьким, то будет использован fallback алгоритм,
|
||||
* так как считается, что тратить время на инициализацию хэш-таблицы не имеет смысла.
|
||||
*/
|
||||
Volnitsky(const char * needle_, size_t needle_size_, size_t haystack_size_hint = 0)
|
||||
: needle(needle_), needle_size(needle_size_), needle_end(needle + needle_size), step(needle_size - sizeof(ngram_t) + 1)
|
||||
VolnitskyBase(const char * const needle, const size_t needle_size, size_t haystack_size_hint = 0)
|
||||
: needle{reinterpret_cast<const UInt8 *>(needle)}, needle_size{needle_size},
|
||||
fallback{
|
||||
needle_size < 2 * sizeof(ngram_t) or needle_size >= std::numeric_limits<offset_t>::max() or
|
||||
(haystack_size_hint and haystack_size_hint < min_haystack_size_for_algorithm)
|
||||
}
|
||||
{
|
||||
if (needle_size < 2 * sizeof(ngram_t)
|
||||
|| needle_size >= std::numeric_limits<offset_t>::max()
|
||||
|| (haystack_size_hint && haystack_size_hint < 20000))
|
||||
{
|
||||
fallback = true;
|
||||
if (fallback)
|
||||
return;
|
||||
}
|
||||
else
|
||||
fallback = false;
|
||||
|
||||
memset(hash, 0, hash_size * sizeof(hash[0]));
|
||||
memset(hash, 0, sizeof(hash));
|
||||
|
||||
for (int i = needle_size - sizeof(ngram_t); i >= 0; --i)
|
||||
{
|
||||
/// Кладём смещение для n-грама в соответствующую ему ячейку или ближайшую свободную.
|
||||
size_t cell_num = *reinterpret_cast<const ngram_t *>(needle + i) % hash_size;
|
||||
while (hash[cell_num])
|
||||
cell_num = (cell_num + 1) % hash_size; /// Поиск следующей свободной ячейки.
|
||||
|
||||
hash[cell_num] = i + 1;
|
||||
}
|
||||
/// int is used here because unsigned can't be used with condition like `i >= 0`, unsigned always >= 0
|
||||
for (auto i = static_cast<int>(needle_size - sizeof(ngram_t)); i >= 0; --i)
|
||||
self().putNGram(this->needle + i, i + 1);
|
||||
}
|
||||
|
||||
|
||||
/// Если не найдено - возвращается конец haystack.
|
||||
const char * search(const char * haystack, size_t haystack_size) const
|
||||
const UInt8 * search(const UInt8 * const haystack, const size_t haystack_size) const
|
||||
{
|
||||
if (needle_size == 0)
|
||||
return haystack;
|
||||
|
||||
const char * haystack_end = haystack + haystack_size;
|
||||
const auto haystack_end = haystack + haystack_size;
|
||||
|
||||
if (needle_size == 1)
|
||||
{
|
||||
const char * res = reinterpret_cast<const char *>(memchr(haystack, needle[0], haystack_size));
|
||||
return res ? res : haystack_end;
|
||||
}
|
||||
if (fallback || haystack_size <= needle_size)
|
||||
{
|
||||
return naive_memmem(haystack, haystack_size, needle, needle_size);
|
||||
}
|
||||
if (needle_size == 1 || fallback || haystack_size <= needle_size)
|
||||
return self().search_fallback(haystack, haystack_end);
|
||||
|
||||
/// Будем "прикладывать" needle к haystack и сравнивать n-грам из конца needle.
|
||||
const char * pos = haystack + needle_size - sizeof(ngram_t);
|
||||
const auto * pos = haystack + needle_size - sizeof(ngram_t);
|
||||
for (; pos <= haystack_end - needle_size; pos += step)
|
||||
{
|
||||
/// Смотрим все ячейки хэш-таблицы, которые могут соответствовать n-граму из haystack.
|
||||
for (size_t cell_num = *reinterpret_cast<const ngram_t *>(pos) % hash_size; hash[cell_num]; cell_num = (cell_num + 1) % hash_size)
|
||||
for (size_t cell_num = toNGram(pos) % hash_size; hash[cell_num];
|
||||
cell_num = (cell_num + 1) % hash_size)
|
||||
{
|
||||
/// Когда нашли - сравниваем побайтово, используя смещение из хэш-таблицы.
|
||||
const char * res = pos - (hash[cell_num] - 1);
|
||||
for (size_t i = 0; i < needle_size; ++i)
|
||||
if (res[i] != needle[i])
|
||||
goto next_hash_cell;
|
||||
const auto res = pos - (hash[cell_num] - 1);
|
||||
|
||||
return res;
|
||||
next_hash_cell:;
|
||||
if (self().compare(res))
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
/// Оставшийся хвостик.
|
||||
return naive_memmem(pos - step + 1, haystack_end - (pos - step + 1), needle, needle_size);
|
||||
return self().search_fallback(pos - step + 1, haystack_end);
|
||||
}
|
||||
|
||||
const unsigned char * search(const unsigned char * haystack, size_t haystack_size) const
|
||||
const char * search(const char * haystack, size_t haystack_size) const
|
||||
{
|
||||
return reinterpret_cast<const unsigned char *>(search(reinterpret_cast<const char *>(haystack), haystack_size));
|
||||
return reinterpret_cast<const char *>(search(reinterpret_cast<const UInt8 *>(haystack), haystack_size));
|
||||
}
|
||||
|
||||
protected:
|
||||
CRTP & self() { return static_cast<CRTP &>(*this); }
|
||||
const CRTP & self() const { return const_cast<VolnitskyBase *>(this)->self(); }
|
||||
|
||||
static const ngram_t & toNGram(const UInt8 * const pos)
|
||||
{
|
||||
return *reinterpret_cast<const ngram_t *>(pos);
|
||||
}
|
||||
|
||||
void putNGramBase(const ngram_t ngram, const int offset)
|
||||
{
|
||||
/// Кладём смещение для n-грама в соответствующую ему ячейку или ближайшую свободную.
|
||||
size_t cell_num = ngram % hash_size;
|
||||
|
||||
while (hash[cell_num])
|
||||
cell_num = (cell_num + 1) % hash_size; /// Поиск следующей свободной ячейки.
|
||||
|
||||
hash[cell_num] = offset;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <bool CaseSensitive, bool ASCII> struct VolnitskyImpl;
|
||||
|
||||
/// Case sensitive comparison
|
||||
template <bool ASCII> struct VolnitskyImpl<true, ASCII> : VolnitskyBase<VolnitskyImpl<true, ASCII>>
|
||||
{
|
||||
VolnitskyImpl(const char * const needle, const size_t needle_size, const size_t haystack_size_hint = 0)
|
||||
: VolnitskyBase<VolnitskyImpl<true, ASCII>>{needle, needle_size, haystack_size_hint},
|
||||
fallback_searcher{needle, needle_size}
|
||||
{
|
||||
}
|
||||
|
||||
void putNGram(const UInt8 * const pos, const int offset)
|
||||
{
|
||||
this->putNGramBase(this->toNGram(pos), offset);
|
||||
}
|
||||
|
||||
bool compare(const UInt8 * const pos) const
|
||||
{
|
||||
/// @todo: maybe just use memcmp for this case and rely on internal SSE optimization as in case with memcpy?
|
||||
return fallback_searcher.compare(pos);
|
||||
}
|
||||
|
||||
const UInt8 * search_fallback(const UInt8 * const haystack, const UInt8 * const haystack_end) const
|
||||
{
|
||||
return fallback_searcher.search(haystack, haystack_end);
|
||||
}
|
||||
|
||||
ASCIICaseSensitiveStringSearcher fallback_searcher;
|
||||
};
|
||||
|
||||
/// Case-insensitive ASCII
|
||||
template <> struct VolnitskyImpl<false, true> : VolnitskyBase<VolnitskyImpl<false, true>>
|
||||
{
|
||||
VolnitskyImpl(const char * const needle, const size_t needle_size, const size_t haystack_size_hint = 0)
|
||||
: VolnitskyBase{needle, needle_size, haystack_size_hint}, fallback_searcher{needle, needle_size}
|
||||
{
|
||||
}
|
||||
|
||||
void putNGram(const UInt8 * const pos, const int offset)
|
||||
{
|
||||
union {
|
||||
ngram_t n;
|
||||
UInt8 c[2];
|
||||
};
|
||||
|
||||
n = toNGram(pos);
|
||||
const auto c0_alpha = std::isalpha(c[0]);
|
||||
const auto c1_alpha = std::isalpha(c[1]);
|
||||
|
||||
if (c0_alpha && c1_alpha)
|
||||
{
|
||||
/// 4 combinations: AB, aB, Ab, ab
|
||||
c[0] = std::tolower(c[0]);
|
||||
c[1] = std::tolower(c[1]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[0] = std::toupper(c[0]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[1] = std::toupper(c[1]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[0] = std::tolower(c[0]);
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
else if (c0_alpha)
|
||||
{
|
||||
/// 2 combinations: A1, a1
|
||||
c[0] = std::tolower(c[0]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[0] = std::toupper(c[0]);
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
else if (c1_alpha)
|
||||
{
|
||||
/// 2 combinations: 0B, 0b
|
||||
c[1] = std::tolower(c[1]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[1] = std::toupper(c[1]);
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
else
|
||||
/// 1 combination: 01
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
|
||||
bool compare(const UInt8 * const pos) const
|
||||
{
|
||||
return fallback_searcher.compare(pos);
|
||||
}
|
||||
|
||||
const UInt8 * search_fallback(const UInt8 * const haystack, const UInt8 * const haystack_end) const
|
||||
{
|
||||
return fallback_searcher.search(haystack, haystack_end);
|
||||
}
|
||||
|
||||
ASCIICaseInsensitiveStringSearcher fallback_searcher;
|
||||
};
|
||||
|
||||
/// Case-sensitive UTF-8
|
||||
template <> struct VolnitskyImpl<false, false> : VolnitskyBase<VolnitskyImpl<false, false>>
|
||||
{
|
||||
VolnitskyImpl(const char * const needle, const size_t needle_size, const size_t haystack_size_hint = 0)
|
||||
: VolnitskyBase{needle, needle_size, haystack_size_hint}, fallback_searcher{needle, needle_size}
|
||||
{
|
||||
}
|
||||
|
||||
void putNGram(const UInt8 * const pos, const int offset)
|
||||
{
|
||||
union
|
||||
{
|
||||
ngram_t n;
|
||||
UInt8 c[2];
|
||||
};
|
||||
|
||||
n = toNGram(pos);
|
||||
|
||||
if (isascii(c[0]) && isascii(c[1]))
|
||||
{
|
||||
const auto c0_al = std::isalpha(c[0]);
|
||||
const auto c1_al = std::isalpha(c[1]);
|
||||
|
||||
if (c0_al && c1_al)
|
||||
{
|
||||
/// 4 combinations: AB, aB, Ab, ab
|
||||
c[0] = std::tolower(c[0]);
|
||||
c[1] = std::tolower(c[1]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[0] = std::toupper(c[0]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[1] = std::toupper(c[1]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[0] = std::tolower(c[0]);
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
else if (c0_al)
|
||||
{
|
||||
/// 2 combinations: A1, a1
|
||||
c[0] = std::tolower(c[0]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[0] = std::toupper(c[0]);
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
else if (c1_al)
|
||||
{
|
||||
/// 2 combinations: 0B, 0b
|
||||
c[1] = std::tolower(c[1]);
|
||||
putNGramBase(n, offset);
|
||||
|
||||
c[1] = std::toupper(c[1]);
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
else
|
||||
/// 1 combination: 01
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
else
|
||||
{
|
||||
using Seq = UInt8[6];
|
||||
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
|
||||
if (UTF8::isContinuationOctet(c[1]))
|
||||
{
|
||||
/// ngram is inside a sequence
|
||||
auto seq_pos = pos;
|
||||
UTF8::syncBackward(seq_pos);
|
||||
|
||||
const auto u32 = utf8.convert(seq_pos);
|
||||
const auto l_u32 = Poco::Unicode::toLower(u32);
|
||||
const auto u_u32 = Poco::Unicode::toUpper(u32);
|
||||
|
||||
/// symbol is case-independent
|
||||
if (l_u32 == u_u32)
|
||||
putNGramBase(n, offset);
|
||||
else
|
||||
{
|
||||
/// where is the given ngram in respect to UTF-8 sequence start?
|
||||
const auto seq_ngram_offset = pos - seq_pos;
|
||||
|
||||
Seq seq;
|
||||
|
||||
/// put ngram from lowercase
|
||||
utf8.convert(l_u32, seq, sizeof(seq));
|
||||
c[0] = seq[seq_ngram_offset];
|
||||
c[1] = seq[seq_ngram_offset + 1];
|
||||
putNGramBase(n, offset);
|
||||
|
||||
/// put ngram for uppercase
|
||||
utf8.convert(u_u32, seq, sizeof(seq));
|
||||
c[0] = seq[seq_ngram_offset];
|
||||
c[1] = seq[seq_ngram_offset + 1];
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// ngram is on the boundary of two sequences
|
||||
/// first sequence may start before u_pos if it is not ASCII
|
||||
auto first_seq_pos = pos;
|
||||
UTF8::syncBackward(first_seq_pos);
|
||||
|
||||
const auto first_u32 = utf8.convert(first_seq_pos);
|
||||
const auto first_l_u32 = Poco::Unicode::toLower(first_u32);
|
||||
const auto first_u_u32 = Poco::Unicode::toUpper(first_u32);
|
||||
|
||||
/// second sequence always start immediately after u_pos
|
||||
auto second_seq_pos = pos + 1;
|
||||
|
||||
const auto second_u32 = utf8.convert(second_seq_pos);
|
||||
const auto second_l_u32 = Poco::Unicode::toLower(second_u32);
|
||||
const auto second_u_u32 = Poco::Unicode::toUpper(second_u32);
|
||||
|
||||
/// both symbols are case-independent
|
||||
if (first_l_u32 == first_u_u32 && second_l_u32 == second_u_u32)
|
||||
putNGramBase(n, offset);
|
||||
else if (first_l_u32 == first_u_u32)
|
||||
{
|
||||
/// first symbol is case-independent
|
||||
Seq seq;
|
||||
|
||||
/// put ngram for lowercase
|
||||
utf8.convert(second_l_u32, seq, sizeof(seq));
|
||||
c[1] = seq[0];
|
||||
putNGramBase(n, offset);
|
||||
|
||||
/// put ngram from uppercase
|
||||
utf8.convert(second_u_u32, seq, sizeof(seq));
|
||||
c[1] = seq[0];
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
else if (second_l_u32 == second_u_u32)
|
||||
{
|
||||
/// second symbol is case-independent
|
||||
|
||||
/// where is the given ngram in respect to the first UTF-8 sequence start?
|
||||
const auto seq_ngram_offset = pos - first_seq_pos;
|
||||
|
||||
Seq seq;
|
||||
|
||||
/// put ngram for lowercase
|
||||
utf8.convert(second_l_u32, seq, sizeof(seq));
|
||||
c[0] = seq[seq_ngram_offset];
|
||||
putNGramBase(n, offset);
|
||||
|
||||
/// put ngram for uppercase
|
||||
utf8.convert(second_u_u32, seq, sizeof(seq));
|
||||
c[0] = seq[seq_ngram_offset];
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// where is the given ngram in respect to the first UTF-8 sequence start?
|
||||
const auto seq_ngram_offset = pos - first_seq_pos;
|
||||
|
||||
Seq first_l_seq, first_u_seq, second_l_seq, second_u_seq;
|
||||
|
||||
utf8.convert(first_l_u32, first_l_seq, sizeof(first_l_seq));
|
||||
utf8.convert(first_u_u32, first_u_seq, sizeof(first_u_seq));
|
||||
utf8.convert(second_l_u32, second_l_seq, sizeof(second_l_seq));
|
||||
utf8.convert(second_u_u32, second_u_seq, sizeof(second_u_seq));
|
||||
|
||||
/// ngram for ll
|
||||
c[0] = first_l_seq[seq_ngram_offset];
|
||||
c[1] = second_l_seq[0];
|
||||
putNGramBase(n, offset);
|
||||
|
||||
/// ngram for lU
|
||||
c[0] = first_l_seq[seq_ngram_offset];
|
||||
c[1] = second_u_seq[0];
|
||||
putNGramBase(n, offset);
|
||||
|
||||
/// ngram for Ul
|
||||
c[0] = first_u_seq[seq_ngram_offset];
|
||||
c[1] = second_l_seq[0];
|
||||
putNGramBase(n, offset);
|
||||
|
||||
/// ngram for UU
|
||||
c[0] = first_u_seq[seq_ngram_offset];
|
||||
c[1] = second_u_seq[0];
|
||||
putNGramBase(n, offset);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool compare(const UInt8 * const pos) const
|
||||
{
|
||||
return fallback_searcher.compare(pos);
|
||||
}
|
||||
|
||||
const UInt8 * search_fallback(const UInt8 * const haystack, const UInt8 * const haystack_end) const
|
||||
{
|
||||
return fallback_searcher.search(haystack, haystack_end);
|
||||
}
|
||||
|
||||
UTF8CaseInsensitiveStringSearcher fallback_searcher;
|
||||
};
|
||||
|
||||
|
||||
using Volnitsky = VolnitskyImpl<true, true>;
|
||||
using VolnitskyUTF8 = VolnitskyImpl<true, false>; /// exactly same as Volnitsky
|
||||
using VolnitskyCaseInsensitive = VolnitskyImpl<false, true>; /// ignores non-ASCII bytes
|
||||
using VolnitskyCaseInsensitiveUTF8 = VolnitskyImpl<false, false>;
|
||||
|
||||
|
||||
}
|
||||
|
@ -130,6 +130,23 @@ typedef std::list<Block> BlocksList;
|
||||
/// Сравнить типы столбцов у блоков. Порядок столбцов имеет значение. Имена не имеют значения.
|
||||
bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs);
|
||||
|
||||
/** Дополнительные данные к блокам. Они пока нужны только для запроса
|
||||
* DESCRIBE TABLE с Distributed-таблицами.
|
||||
*/
|
||||
struct BlockExtraInfo
|
||||
{
|
||||
BlockExtraInfo() {}
|
||||
operator bool() const { return is_valid; }
|
||||
bool operator!() const { return !is_valid; }
|
||||
|
||||
std::string host;
|
||||
std::string resolved_address;
|
||||
std::string user;
|
||||
UInt16 port = 0;
|
||||
|
||||
bool is_valid = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
namespace std
|
||||
|
@ -295,6 +295,10 @@ namespace ErrorCodes
|
||||
DATABASE_ACCESS_DENIED = 291,
|
||||
LEADERSHIP_CHANGED = 292,
|
||||
MONGODB_INIT_FAILED = 293,
|
||||
INVALID_BLOCK_EXTRA_INFO = 294,
|
||||
RECEIVED_EMPTY_DATA = 295,
|
||||
NO_REMOTE_SHARD_FOUND = 296,
|
||||
SHARD_HAS_NO_CONNECTIONS = 297,
|
||||
|
||||
KEEPER_EXCEPTION = 999,
|
||||
POCO_EXCEPTION = 1000,
|
||||
|
44
dbms/include/DB/DataStreams/BlockExtraInfoInputStream.h
Normal file
44
dbms/include/DB/DataStreams/BlockExtraInfoInputStream.h
Normal file
@ -0,0 +1,44 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Прибавляет к одному потоку дополнительную информацию о блоках, которая задана
|
||||
* в качестве параметра конструктора.
|
||||
*/
|
||||
class BlockExtraInfoInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
BlockExtraInfoInputStream(BlockInputStreamPtr input_, const BlockExtraInfo & block_extra_info_)
|
||||
: block_extra_info(block_extra_info_)
|
||||
{
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
BlockExtraInfo getBlockExtraInfo() const override
|
||||
{
|
||||
return block_extra_info;
|
||||
}
|
||||
|
||||
String getName() const override { return "BlockExtraInfoInput"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
std::stringstream res;
|
||||
res << "BlockExtraInfoInput(" << children.back()->getID() << ")";
|
||||
return res.str();
|
||||
}
|
||||
|
||||
protected:
|
||||
Block readImpl() override
|
||||
{
|
||||
return children.back()->read();
|
||||
}
|
||||
|
||||
private:
|
||||
BlockExtraInfo block_extra_info;
|
||||
};
|
||||
|
||||
}
|
@ -38,6 +38,13 @@ public:
|
||||
*/
|
||||
virtual Block read() = 0;
|
||||
|
||||
/** Получить информацию про последний полученный блок.
|
||||
*/
|
||||
virtual BlockExtraInfo getBlockExtraInfo() const
|
||||
{
|
||||
throw Exception("Method getBlockExtraInfo is not supported by the data stream " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/** Прочитать что-нибудь перед началом всех данных или после конца всех данных.
|
||||
* В функции readSuffix можно реализовать финализацию, которая может привести к исключению.
|
||||
* readPrefix() должна вызываться до первого вызова read().
|
||||
|
@ -26,6 +26,13 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Режим объединения.
|
||||
*/
|
||||
enum class StreamUnionMode
|
||||
{
|
||||
Basic = 0, /// вынимать блоки
|
||||
ExtraInfo /// вынимать блоки + дополнительную информацию
|
||||
};
|
||||
|
||||
/// Пример обработчика.
|
||||
struct ParallelInputsHandler
|
||||
@ -33,6 +40,9 @@ struct ParallelInputsHandler
|
||||
/// Обработка блока данных.
|
||||
void onBlock(Block & block, size_t thread_num) {}
|
||||
|
||||
/// Обработка блока данных + дополнительных информаций.
|
||||
void onBlock(Block & block, BlockExtraInfo & extra_info, size_t thread_num) {}
|
||||
|
||||
/// Блоки закончились. Из-за того, что все источники иссякли или из-за отмены работы.
|
||||
/// Этот метод всегда вызывается ровно один раз, в конце работы, если метод onException не кидает исключение.
|
||||
void onFinish() {}
|
||||
@ -42,7 +52,7 @@ struct ParallelInputsHandler
|
||||
};
|
||||
|
||||
|
||||
template <typename Handler>
|
||||
template <typename Handler, StreamUnionMode mode = StreamUnionMode::Basic>
|
||||
class ParallelInputsProcessor
|
||||
{
|
||||
public:
|
||||
@ -136,6 +146,20 @@ private:
|
||||
InputData(BlockInputStreamPtr & in_, size_t i_) : in(in_), i(i_) {}
|
||||
};
|
||||
|
||||
template <StreamUnionMode mode2 = mode>
|
||||
void publishPayload(BlockInputStreamPtr & stream, Block & block, size_t thread_num,
|
||||
typename std::enable_if<mode2 == StreamUnionMode::Basic>::type * = nullptr)
|
||||
{
|
||||
handler.onBlock(block, thread_num);
|
||||
}
|
||||
|
||||
template <StreamUnionMode mode2 = mode>
|
||||
void publishPayload(BlockInputStreamPtr & stream, Block & block, size_t thread_num,
|
||||
typename std::enable_if<mode2 == StreamUnionMode::ExtraInfo>::type * = nullptr)
|
||||
{
|
||||
BlockExtraInfo extra_info = stream->getBlockExtraInfo();
|
||||
handler.onBlock(block, extra_info, thread_num);
|
||||
}
|
||||
|
||||
void thread(MemoryTracker * memory_tracker, size_t thread_num)
|
||||
{
|
||||
@ -167,7 +191,7 @@ private:
|
||||
try
|
||||
{
|
||||
while (Block block = additional_input_at_end->read())
|
||||
handler.onBlock(block, thread_num);
|
||||
publishPayload(additional_input_at_end, block, thread_num);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -230,7 +254,7 @@ private:
|
||||
break;
|
||||
|
||||
if (block)
|
||||
handler.onBlock(block, thread_num);
|
||||
publishPayload(input.in, block, thread_num);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -119,6 +119,23 @@ public:
|
||||
sendQuery();
|
||||
}
|
||||
|
||||
/// Отправить запрос на все существующие реплики.
|
||||
void reachAllReplicas()
|
||||
{
|
||||
reach_all_replicas = true;
|
||||
}
|
||||
|
||||
/// Кроме блоков, получить информацию о блоках.
|
||||
void appendExtraInfo()
|
||||
{
|
||||
append_extra_info = true;
|
||||
}
|
||||
|
||||
BlockExtraInfo getBlockExtraInfo() const override
|
||||
{
|
||||
return parallel_replicas->getBlockExtraInfo();
|
||||
}
|
||||
|
||||
protected:
|
||||
/// Отправить на удаленные серверы все временные таблицы.
|
||||
void sendExternalTables()
|
||||
@ -264,7 +281,8 @@ protected:
|
||||
if (connection != nullptr)
|
||||
parallel_replicas = std::make_unique<ParallelReplicas>(connection, parallel_replicas_settings, throttler);
|
||||
else
|
||||
parallel_replicas = std::make_unique<ParallelReplicas>(pool, parallel_replicas_settings, throttler);
|
||||
parallel_replicas = std::make_unique<ParallelReplicas>(pool, parallel_replicas_settings, throttler,
|
||||
append_extra_info, reach_all_replicas);
|
||||
}
|
||||
|
||||
/// Возвращает true, если запрос отправлен.
|
||||
@ -365,6 +383,9 @@ private:
|
||||
*/
|
||||
std::atomic<bool> got_unknown_packet_from_replica { false };
|
||||
|
||||
bool append_extra_info = false;
|
||||
bool reach_all_replicas = false;
|
||||
|
||||
Logger * log = &Logger::get("RemoteBlockInputStream");
|
||||
};
|
||||
|
||||
|
@ -12,6 +12,38 @@ namespace DB
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <StreamUnionMode mode>
|
||||
struct OutputData;
|
||||
|
||||
/// Блок или эксепшен.
|
||||
template <>
|
||||
struct OutputData<StreamUnionMode::Basic>
|
||||
{
|
||||
Block block;
|
||||
std::exception_ptr exception;
|
||||
|
||||
OutputData() {}
|
||||
OutputData(Block & block_) : block(block_) {}
|
||||
OutputData(std::exception_ptr & exception_) : exception(exception_) {}
|
||||
};
|
||||
|
||||
/// Блок + дополнительнцю информацию или эксепшен.
|
||||
template <>
|
||||
struct OutputData<StreamUnionMode::ExtraInfo>
|
||||
{
|
||||
Block block;
|
||||
BlockExtraInfo extra_info;
|
||||
std::exception_ptr exception;
|
||||
|
||||
OutputData() {}
|
||||
OutputData(Block & block_, BlockExtraInfo & extra_info_) : block(block_), extra_info(extra_info_) {}
|
||||
OutputData(std::exception_ptr & exception_) : exception(exception_) {}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
/** Объединяет несколько источников в один.
|
||||
* Блоки из разных источников перемежаются друг с другом произвольным образом.
|
||||
@ -21,12 +53,17 @@ using Poco::SharedPtr;
|
||||
* Устроено так:
|
||||
* - с помощью ParallelInputsProcessor в нескольких потоках вынимает из источников блоки;
|
||||
* - полученные блоки складываются в ограниченную очередь готовых блоков;
|
||||
* - основной поток вынимает готовые блоки из очереди готовых блоков.
|
||||
* - основной поток вынимает готовые блоки из очереди готовых блоков;
|
||||
* - если указан режим StreamUnionMode::ExtraInfo, в дополнение к блокам UnionBlockInputStream
|
||||
* вынимает информацию о блоках; в таком случае все источники должны поддержать такой режим.
|
||||
*/
|
||||
|
||||
|
||||
template <StreamUnionMode mode = StreamUnionMode::Basic>
|
||||
class UnionBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
private:
|
||||
using Self = UnionBlockInputStream<mode>;
|
||||
|
||||
public:
|
||||
UnionBlockInputStream(BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end, size_t max_threads) :
|
||||
output_queue(std::min(inputs.size(), max_threads)),
|
||||
@ -88,6 +125,10 @@ public:
|
||||
processor.cancel();
|
||||
}
|
||||
|
||||
BlockExtraInfo getBlockExtraInfo() const override
|
||||
{
|
||||
return doGetBlockExtraInfo();
|
||||
}
|
||||
|
||||
protected:
|
||||
void finalize()
|
||||
@ -103,7 +144,7 @@ protected:
|
||||
/** Прочитаем всё до конца, чтобы ParallelInputsProcessor не заблокировался при попытке вставить в очередь.
|
||||
* Может быть, в очереди есть ещё эксепшен.
|
||||
*/
|
||||
OutputData res;
|
||||
OutputData<mode> res;
|
||||
while (true)
|
||||
{
|
||||
//std::cerr << "popping\n";
|
||||
@ -141,9 +182,8 @@ protected:
|
||||
|
||||
Block readImpl() override
|
||||
{
|
||||
OutputData res;
|
||||
if (all_read)
|
||||
return res.block;
|
||||
return received_payload.block;
|
||||
|
||||
/// Запускаем потоки, если это ещё не было сделано.
|
||||
if (!started)
|
||||
@ -154,15 +194,15 @@ protected:
|
||||
|
||||
/// Будем ждать, пока будет готов следующий блок или будет выкинуто исключение.
|
||||
//std::cerr << "popping\n";
|
||||
output_queue.pop(res);
|
||||
output_queue.pop(received_payload);
|
||||
|
||||
if (res.exception)
|
||||
std::rethrow_exception(res.exception);
|
||||
if (received_payload.exception)
|
||||
std::rethrow_exception(received_payload.exception);
|
||||
|
||||
if (!res.block)
|
||||
if (!received_payload.block)
|
||||
all_read = true;
|
||||
|
||||
return res.block;
|
||||
return received_payload.block;
|
||||
}
|
||||
|
||||
/// Вызывается либо после того, как всё прочитано, либо после cancel-а.
|
||||
@ -179,41 +219,56 @@ protected:
|
||||
}
|
||||
|
||||
private:
|
||||
/// Блок или эксепшен.
|
||||
struct OutputData
|
||||
template<StreamUnionMode mode2 = mode>
|
||||
BlockExtraInfo doGetBlockExtraInfo(typename std::enable_if<mode2 == StreamUnionMode::ExtraInfo>::type * = nullptr) const
|
||||
{
|
||||
Block block;
|
||||
std::exception_ptr exception;
|
||||
return received_payload.extra_info;
|
||||
}
|
||||
|
||||
OutputData() {}
|
||||
OutputData(Block & block_) : block(block_) {}
|
||||
OutputData(std::exception_ptr & exception_) : exception(exception_) {}
|
||||
};
|
||||
template<StreamUnionMode mode2 = mode>
|
||||
BlockExtraInfo doGetBlockExtraInfo(typename std::enable_if<mode2 == StreamUnionMode::Basic>::type * = nullptr) const
|
||||
{
|
||||
throw Exception("Method getBlockExtraInfo is not supported for mode StreamUnionMode::Basic",
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
private:
|
||||
using Payload = OutputData<mode>;
|
||||
using OutputQueue = ConcurrentBoundedQueue<Payload>;
|
||||
|
||||
private:
|
||||
/** Очередь готовых блоков. Также туда можно положить эксепшен вместо блока.
|
||||
* Когда данные закончатся - в очередь вставляется пустой блок.
|
||||
* В очередь всегда (даже после исключения или отмены запроса) рано или поздно вставляется пустой блок.
|
||||
* Очередь всегда (даже после исключения или отмены запроса, даже в деструкторе) нужно дочитывать до пустого блока,
|
||||
* иначе ParallelInputsProcessor может заблокироваться при вставке в очередь.
|
||||
*/
|
||||
typedef ConcurrentBoundedQueue<OutputData> OutputQueue;
|
||||
OutputQueue output_queue;
|
||||
|
||||
|
||||
struct Handler
|
||||
{
|
||||
Handler(UnionBlockInputStream & parent_) : parent(parent_) {}
|
||||
Handler(Self & parent_) : parent(parent_) {}
|
||||
|
||||
void onBlock(Block & block, size_t thread_num)
|
||||
template <StreamUnionMode mode2 = mode>
|
||||
void onBlock(Block & block, size_t thread_num,
|
||||
typename std::enable_if<mode2 == StreamUnionMode::Basic>::type * = nullptr)
|
||||
{
|
||||
//std::cerr << "pushing block\n";
|
||||
parent.output_queue.push(block);
|
||||
parent.output_queue.push(Payload(block));
|
||||
}
|
||||
|
||||
template <StreamUnionMode mode2 = mode>
|
||||
void onBlock(Block & block, BlockExtraInfo & extra_info, size_t thread_num,
|
||||
typename std::enable_if<mode2 == StreamUnionMode::ExtraInfo>::type * = nullptr)
|
||||
{
|
||||
//std::cerr << "pushing block with extra info\n";
|
||||
parent.output_queue.push(Payload(block, extra_info));
|
||||
}
|
||||
|
||||
void onFinish()
|
||||
{
|
||||
//std::cerr << "pushing end\n";
|
||||
parent.output_queue.push(OutputData());
|
||||
parent.output_queue.push(Payload());
|
||||
}
|
||||
|
||||
void onException(std::exception_ptr & exception, size_t thread_num)
|
||||
@ -228,11 +283,13 @@ private:
|
||||
parent.cancel(); /// Не кидает исключений.
|
||||
}
|
||||
|
||||
UnionBlockInputStream & parent;
|
||||
Self & parent;
|
||||
};
|
||||
|
||||
Handler handler;
|
||||
ParallelInputsProcessor<Handler> processor;
|
||||
ParallelInputsProcessor<Handler, mode> processor;
|
||||
|
||||
Payload received_payload;
|
||||
|
||||
bool started = false;
|
||||
bool all_read = false;
|
||||
|
@ -5,10 +5,12 @@
|
||||
#include <DB/Dictionaries/FileDictionarySource.h>
|
||||
#include <DB/Dictionaries/MySQLDictionarySource.h>
|
||||
#include <DB/Dictionaries/ClickHouseDictionarySource.h>
|
||||
//#include <DB/Dictionaries/MongoDBDictionarySource.h>
|
||||
#include <DB/Dictionaries/MongoDBDictionarySource.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <common/singleton.h>
|
||||
#include <memory>
|
||||
#include <DB/Core/FieldVisitors.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -19,25 +21,21 @@ namespace
|
||||
Block createSampleBlock(const DictionaryStructure & dict_struct)
|
||||
{
|
||||
Block block{
|
||||
ColumnWithTypeAndName{
|
||||
new ColumnUInt64,
|
||||
new DataTypeUInt64,
|
||||
dict_struct.id.name
|
||||
}
|
||||
ColumnWithTypeAndName{new ColumnUInt64{1}, new DataTypeUInt64, dict_struct.id.name}
|
||||
};
|
||||
|
||||
if (dict_struct.range_min)
|
||||
for (const auto & attribute : { dict_struct.range_min, dict_struct.range_max })
|
||||
block.insert(ColumnWithTypeAndName{
|
||||
new ColumnUInt16,
|
||||
new DataTypeDate,
|
||||
attribute->name
|
||||
});
|
||||
block.insert(
|
||||
ColumnWithTypeAndName{new ColumnUInt16{1}, new DataTypeDate, attribute->name});
|
||||
|
||||
for (const auto & attribute : dict_struct.attributes)
|
||||
block.insert(ColumnWithTypeAndName{
|
||||
attribute.type->createColumn(), attribute.type, attribute.name
|
||||
});
|
||||
{
|
||||
auto column = attribute.type->createColumn();
|
||||
column->insert(attribute.null_value);
|
||||
|
||||
block.insert(ColumnWithTypeAndName{column, attribute.type, attribute.name});
|
||||
}
|
||||
|
||||
return block;
|
||||
}
|
||||
@ -85,11 +83,11 @@ public:
|
||||
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse",
|
||||
sample_block, context);
|
||||
}
|
||||
/* else if ("mongodb" == source_type)
|
||||
else if ("mongodb" == source_type)
|
||||
{
|
||||
return std::make_unique<MongoDBDictionarySource>(dict_struct, config, config_prefix + ".mongodb",
|
||||
sample_block, context);
|
||||
}*/
|
||||
}
|
||||
|
||||
throw Exception{
|
||||
name + ": unknown dictionary source type: " + source_type,
|
||||
|
@ -267,7 +267,7 @@ private:
|
||||
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
|
||||
case AttributeUnderlyingType::String:
|
||||
{
|
||||
const auto & null_value_ref = std::get<String>(attr.null_values) = null_value.get<String>();
|
||||
std::get<String>(attr.null_values) = null_value.get<String>();
|
||||
std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attr.maps) =
|
||||
std::make_unique<HashMap<UInt64, StringRef>>();
|
||||
attr.string_arena = std::make_unique<Arena>();
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
@ -11,11 +12,13 @@
|
||||
#include <mongo/client/dbclient.h>
|
||||
#include <vector>
|
||||
#include <string>
|
||||
#include <DB/Core/FieldVisitors.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Allows processing results of a MongoDB query as a sequence of Blocks, simplifies chaining
|
||||
/// Converts mongo::DBClientCursor to a stream of DB::Block`s
|
||||
class MongoDBBlockInputStream final : public IProfilingBlockInputStream
|
||||
{
|
||||
enum struct value_type_t
|
||||
@ -37,15 +40,20 @@ class MongoDBBlockInputStream final : public IProfilingBlockInputStream
|
||||
|
||||
public:
|
||||
MongoDBBlockInputStream(
|
||||
std::unique_ptr<mongo::DBClientCursor> cursor_, const Block & sample_block, const std::size_t max_block_size)
|
||||
: cursor{std::move(cursor_)}, sample_block{sample_block}, max_block_size{max_block_size}
|
||||
std::unique_ptr<mongo::DBClientCursor> cursor_, const Block & sample_block_, const std::size_t max_block_size)
|
||||
: cursor{std::move(cursor_)}, sample_block{sample_block_}, max_block_size{max_block_size}
|
||||
{
|
||||
/// do nothing if cursor has no data
|
||||
if (!cursor->more())
|
||||
return;
|
||||
|
||||
types.reserve(sample_block.columns());
|
||||
const auto num_columns = sample_block.columns();
|
||||
types.reserve(num_columns);
|
||||
names.reserve(num_columns);
|
||||
sample_columns.reserve(num_columns);
|
||||
|
||||
for (const auto idx : ext::range(0, sample_block.columns()))
|
||||
/// save types of each column to eliminate subsequent typeid_cast<> invocations
|
||||
for (const auto idx : ext::range(0, num_columns))
|
||||
{
|
||||
const auto & column = sample_block.getByPosition(idx);
|
||||
const auto type = column.type.get();
|
||||
@ -83,6 +91,7 @@ public:
|
||||
};
|
||||
|
||||
names.emplace_back(column.name);
|
||||
sample_columns.emplace_back(column.column.get());
|
||||
}
|
||||
}
|
||||
|
||||
@ -98,6 +107,7 @@ public:
|
||||
private:
|
||||
Block readImpl() override
|
||||
{
|
||||
/// return an empty block if cursor has no data
|
||||
if (!cursor->more())
|
||||
return {};
|
||||
|
||||
@ -121,7 +131,7 @@ private:
|
||||
if (value.ok())
|
||||
insertValue(columns[idx], types[idx], value);
|
||||
else
|
||||
insertDefaultValue(columns[idx], types[idx]);
|
||||
insertDefaultValue(columns[idx], *sample_columns[idx]);
|
||||
}
|
||||
|
||||
++num_rows;
|
||||
@ -284,25 +294,9 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
/// @todo insert default value from the dictionary attribute definition
|
||||
static void insertDefaultValue(IColumn * const column, const value_type_t type)
|
||||
static void insertDefaultValue(IColumn * const column, const IColumn & sample_column)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case value_type_t::UInt8: static_cast<ColumnUInt8 *>(column)->insertDefault(); break;
|
||||
case value_type_t::UInt16: static_cast<ColumnUInt16 *>(column)->insertDefault(); break;
|
||||
case value_type_t::UInt32: static_cast<ColumnUInt32 *>(column)->insertDefault(); break;
|
||||
case value_type_t::UInt64: static_cast<ColumnUInt64 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Int8: static_cast<ColumnInt8 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Int16: static_cast<ColumnInt16 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Int32: static_cast<ColumnInt32 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Int64: static_cast<ColumnInt64 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Float32: static_cast<ColumnFloat32 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Float64: static_cast<ColumnFloat64 *>(column)->insertDefault(); break;
|
||||
case value_type_t::String: static_cast<ColumnString *>(column)->insertDefault(); break;
|
||||
case value_type_t::Date: static_cast<ColumnUInt16 *>(column)->insertDefault(); break;
|
||||
case value_type_t::DateTime: static_cast<ColumnUInt32 *>(column)->insertDefault(); break;
|
||||
}
|
||||
column->insertFrom(sample_column, 0);
|
||||
}
|
||||
|
||||
std::unique_ptr<mongo::DBClientCursor> cursor;
|
||||
@ -310,6 +304,7 @@ private:
|
||||
const std::size_t max_block_size;
|
||||
std::vector<value_type_t> types;
|
||||
std::vector<mongo::StringData> names;
|
||||
std::vector<const IColumn *> sample_columns;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,7 +9,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Allows loading dictionaries from a MySQL database
|
||||
/// Allows loading dictionaries from a MongoDB collection
|
||||
class MongoDBDictionarySource final : public IDictionarySource
|
||||
{
|
||||
MongoDBDictionarySource(
|
||||
@ -25,6 +25,7 @@ class MongoDBDictionarySource final : public IDictionarySource
|
||||
|
||||
connection.connect(host + ':' + port);
|
||||
|
||||
/// @todo: should connection.auth be called after or before .connect ?
|
||||
if (!user.empty())
|
||||
{
|
||||
std::string error;
|
||||
@ -45,6 +46,7 @@ class MongoDBDictionarySource final : public IDictionarySource
|
||||
fields_to_query = builder.obj();
|
||||
}
|
||||
|
||||
/// mongo-cxx driver requires global initialization before using any functionality
|
||||
static void init()
|
||||
{
|
||||
static const auto mongo_init_status = mongo::client::initialize();
|
||||
@ -95,12 +97,17 @@ public:
|
||||
|
||||
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
|
||||
{
|
||||
/// mongo::BSONObj has shitty design and does not use fixed width integral types
|
||||
const std::vector<long long int> iids{std::begin(ids), std::end(ids)};
|
||||
const auto ids_enumeration = BSON(dict_struct.id.name << BSON("$in" << iids));
|
||||
|
||||
return new MongoDBBlockInputStream{
|
||||
connection.query(db + '.' + collection, {}, 0, 0, &fields_to_query),
|
||||
connection.query(db + '.' + collection, ids_enumeration, 0, 0, &fields_to_query),
|
||||
sample_block, 8192
|
||||
};
|
||||
}
|
||||
|
||||
/// @todo: for MongoDB, modification date can somehow be determined from the `_id` object field
|
||||
bool isModified() const override { return false; }
|
||||
|
||||
DictionarySourcePtr clone() const override { return std::make_unique<MongoDBDictionarySource>(*this); }
|
||||
|
@ -37,12 +37,11 @@ class MySQLBlockInputStream final : public IProfilingBlockInputStream
|
||||
};
|
||||
|
||||
public:
|
||||
MySQLBlockInputStream(const mysqlxx::PoolWithFailover::Entry & entry,
|
||||
const std::string & query_str,
|
||||
const Block & sample_block,
|
||||
MySQLBlockInputStream(
|
||||
const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block_,
|
||||
const std::size_t max_block_size)
|
||||
: entry{entry}, query{this->entry->query(query_str)}, result{query.use()},
|
||||
sample_block{sample_block}, max_block_size{max_block_size}
|
||||
sample_block{sample_block_}, max_block_size{max_block_size}
|
||||
{
|
||||
if (sample_block.columns() != result.getNumFields())
|
||||
throw Exception{
|
||||
@ -51,11 +50,15 @@ public:
|
||||
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH
|
||||
};
|
||||
|
||||
types.reserve(sample_block.columns());
|
||||
const auto num_columns = sample_block.columns();
|
||||
types.reserve(num_columns);
|
||||
sample_columns.reserve(num_columns);
|
||||
|
||||
for (const auto idx : ext::range(0, sample_block.columns()))
|
||||
for (const auto idx : ext::range(0, num_columns))
|
||||
{
|
||||
const auto type = sample_block.getByPosition(idx).type.get();
|
||||
const auto & column = sample_block.getByPosition(idx);
|
||||
const auto type = column.type.get();
|
||||
|
||||
if (typeid_cast<const DataTypeUInt8 *>(type))
|
||||
types.push_back(value_type_t::UInt8);
|
||||
else if (typeid_cast<const DataTypeUInt16 *>(type))
|
||||
@ -87,6 +90,8 @@ public:
|
||||
"Unsupported type " + type->getName(),
|
||||
ErrorCodes::UNKNOWN_TYPE
|
||||
};
|
||||
|
||||
sample_columns.emplace_back(column.column.get());
|
||||
}
|
||||
}
|
||||
|
||||
@ -120,7 +125,7 @@ private:
|
||||
if (!value.isNull())
|
||||
insertValue(columns[idx], types[idx], value);
|
||||
else
|
||||
insertDefaultValue(columns[idx], types[idx]);
|
||||
insertDefaultValue(columns[idx], *sample_columns[idx]);
|
||||
}
|
||||
|
||||
++num_rows;
|
||||
@ -158,24 +163,9 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
static void insertDefaultValue(IColumn * const column, const value_type_t type)
|
||||
static void insertDefaultValue(IColumn * const column, const IColumn & sample_column)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case value_type_t::UInt8: static_cast<ColumnUInt8 *>(column)->insertDefault(); break;
|
||||
case value_type_t::UInt16: static_cast<ColumnUInt16 *>(column)->insertDefault(); break;
|
||||
case value_type_t::UInt32: static_cast<ColumnUInt32 *>(column)->insertDefault(); break;
|
||||
case value_type_t::UInt64: static_cast<ColumnUInt64 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Int8: static_cast<ColumnInt8 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Int16: static_cast<ColumnInt16 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Int32: static_cast<ColumnInt32 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Int64: static_cast<ColumnInt64 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Float32: static_cast<ColumnFloat32 *>(column)->insertDefault(); break;
|
||||
case value_type_t::Float64: static_cast<ColumnFloat64 *>(column)->insertDefault(); break;
|
||||
case value_type_t::String: static_cast<ColumnString *>(column)->insertDefault(); break;
|
||||
case value_type_t::Date: static_cast<ColumnUInt16 *>(column)->insertDefault(); break;
|
||||
case value_type_t::DateTime: static_cast<ColumnUInt32 *>(column)->insertDefault(); break;
|
||||
}
|
||||
column->insertFrom(sample_column, 0);
|
||||
}
|
||||
|
||||
mysqlxx::PoolWithFailover::Entry entry;
|
||||
@ -184,6 +174,7 @@ private:
|
||||
Block sample_block;
|
||||
const std::size_t max_block_size;
|
||||
std::vector<value_type_t> types;
|
||||
std::vector<const IColumn *> sample_columns;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -289,7 +289,7 @@ private:
|
||||
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
|
||||
case AttributeUnderlyingType::String:
|
||||
{
|
||||
const auto & null_value_ref = std::get<String>(attr.null_values) = null_value.get<String>();
|
||||
std::get<String>(attr.null_values) = null_value.get<String>();
|
||||
std::get<ptr_t<StringRef>>(attr.maps) = std::make_unique<collection_t<StringRef>>();
|
||||
attr.string_arena = std::make_unique<Arena>();
|
||||
break;
|
||||
|
@ -2101,6 +2101,225 @@ private:
|
||||
};
|
||||
|
||||
|
||||
class FunctionArrayReverse : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "reverse";
|
||||
static IFunction * create(const Context & context) { return new FunctionArrayReverse; }
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("Argument for function " + getName() + " must be array.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0]->clone();
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (executeConst(block, arguments, result))
|
||||
return;
|
||||
|
||||
const ColumnArray * array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
if (!array)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
ColumnPtr res_ptr = array->cloneEmpty();
|
||||
block.getByPosition(result).column = res_ptr;
|
||||
ColumnArray & res = static_cast<ColumnArray &>(*res_ptr);
|
||||
|
||||
const IColumn & src_data = array->getData();
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
IColumn & res_data = res.getData();
|
||||
res.getOffsetsColumn() = array->getOffsetsColumn();
|
||||
|
||||
if (!( executeNumber<UInt8> (src_data, offsets, res_data)
|
||||
|| executeNumber<UInt16> (src_data, offsets, res_data)
|
||||
|| executeNumber<UInt32> (src_data, offsets, res_data)
|
||||
|| executeNumber<UInt64> (src_data, offsets, res_data)
|
||||
|| executeNumber<Int8> (src_data, offsets, res_data)
|
||||
|| executeNumber<Int16> (src_data, offsets, res_data)
|
||||
|| executeNumber<Int32> (src_data, offsets, res_data)
|
||||
|| executeNumber<Int64> (src_data, offsets, res_data)
|
||||
|| executeNumber<Float32> (src_data, offsets, res_data)
|
||||
|| executeNumber<Float64> (src_data, offsets, res_data)
|
||||
|| executeString (src_data, offsets, res_data)
|
||||
|| executeFixedString (src_data, offsets, res_data)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (const ColumnConstArray * const_array = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
const Array & arr = const_array->getData();
|
||||
|
||||
size_t size = arr.size();
|
||||
Array res(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
res[i] = arr[size - i - 1];
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstArray(
|
||||
block.rowsInFirstColumn(),
|
||||
res,
|
||||
block.getByPosition(arguments[0]).type->clone());
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(
|
||||
const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets,
|
||||
IColumn & res_data_col)
|
||||
{
|
||||
if (const ColumnVector<T> * src_data_concrete = typeid_cast<const ColumnVector<T> *>(&src_data))
|
||||
{
|
||||
const PODArray<T> & src_data = src_data_concrete->getData();
|
||||
PODArray<T> & res_data = typeid_cast<ColumnVector<T> &>(res_data_col).getData();
|
||||
size_t size = src_offsets.size();
|
||||
res_data.resize(src_data.size());
|
||||
|
||||
ColumnArray::Offset_t src_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const T * src = &src_data[src_prev_offset];
|
||||
const T * src_end = &src_data[src_offsets[i]];
|
||||
|
||||
if (src == src_end)
|
||||
continue;
|
||||
|
||||
T * dst = &res_data[src_offsets[i] - 1];
|
||||
|
||||
while (src < src_end)
|
||||
{
|
||||
*dst = *src;
|
||||
++src;
|
||||
--dst;
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
bool executeFixedString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets,
|
||||
IColumn & res_data_col)
|
||||
{
|
||||
if (const ColumnFixedString * src_data_concrete = typeid_cast<const ColumnFixedString *>(&src_data))
|
||||
{
|
||||
const size_t n = src_data_concrete->getN();
|
||||
const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars();
|
||||
ColumnFixedString::Chars_t & res_data = typeid_cast<ColumnFixedString &>(res_data_col).getChars();
|
||||
size_t size = src_offsets.size();
|
||||
res_data.resize(src_data.size());
|
||||
|
||||
ColumnArray::Offset_t src_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 * src = &src_data[src_prev_offset * n];
|
||||
const UInt8 * src_end = &src_data[src_offsets[i] * n];
|
||||
|
||||
if (src == src_end)
|
||||
continue;
|
||||
|
||||
UInt8 * dst = &res_data[src_offsets[i] * n - n];
|
||||
|
||||
while (src < src_end)
|
||||
{
|
||||
memcpy(dst, src, n);
|
||||
src += n;
|
||||
dst -= n;
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
bool executeString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets_t & src_array_offsets,
|
||||
IColumn & res_data_col)
|
||||
{
|
||||
if (const ColumnString * src_data_concrete = typeid_cast<const ColumnString *>(&src_data))
|
||||
{
|
||||
const ColumnString::Offsets_t & src_string_offsets = src_data_concrete->getOffsets();
|
||||
ColumnString::Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res_data_col).getOffsets();
|
||||
|
||||
const ColumnString::Chars_t & src_data = src_data_concrete->getChars();
|
||||
ColumnString::Chars_t & res_data = typeid_cast<ColumnString &>(res_data_col).getChars();
|
||||
|
||||
size_t size = src_array_offsets.size();
|
||||
res_string_offsets.resize(src_string_offsets.size());
|
||||
res_data.resize(src_data.size());
|
||||
|
||||
ColumnArray::Offset_t src_array_prev_offset = 0;
|
||||
ColumnString::Offset_t res_string_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (src_array_offsets[i] != src_array_prev_offset)
|
||||
{
|
||||
size_t array_size = src_array_offsets[i] - src_array_prev_offset;
|
||||
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
size_t j_reversed = array_size - j - 1;
|
||||
|
||||
auto src_pos = src_array_prev_offset + j_reversed == 0 ? 0 : src_string_offsets[src_array_prev_offset + j_reversed - 1];
|
||||
size_t string_size = src_string_offsets[src_array_prev_offset + j_reversed] - src_pos;
|
||||
|
||||
memcpy(&res_data[res_string_prev_offset], &src_data[src_pos], string_size);
|
||||
|
||||
res_string_prev_offset += string_size;
|
||||
res_string_offsets[src_array_prev_offset + j] = res_string_prev_offset;
|
||||
}
|
||||
}
|
||||
|
||||
src_array_prev_offset = src_array_offsets[i];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NameHas { static constexpr auto name = "has"; };
|
||||
struct NameIndexOf { static constexpr auto name = "indexOf"; };
|
||||
struct NameCountEqual { static constexpr auto name = "countEqual"; };
|
||||
|
@ -97,43 +97,195 @@ struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
||||
}
|
||||
};
|
||||
|
||||
/// Реализация функции toDate.
|
||||
|
||||
/** Преобразование даты-с-временем в дату: отбрасывание времени.
|
||||
*/
|
||||
template <typename Name>
|
||||
struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
||||
namespace details { namespace {
|
||||
|
||||
template<typename FromType, typename ToType, template <typename, typename> class Transformation>
|
||||
class Transformer
|
||||
{
|
||||
typedef DataTypeDateTime::FieldType FromFieldType;
|
||||
typedef DataTypeDate::FieldType ToFieldType;
|
||||
private:
|
||||
using Op = Transformation<FromType, ToType>;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
public:
|
||||
static void vector_vector(const PODArray<FromType> & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToType> & vec_to)
|
||||
{
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
{
|
||||
ColumnVector<ToFieldType> * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<FromFieldType>::Container_t & vec_from = col_from->getData();
|
||||
typename ColumnVector<ToFieldType>::Container_t & vec_to = col_to->getData();
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = date_lut.toDayNum(vec_from[i]);
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Op::execute(vec_from[i], remote_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
else if (const ColumnConst<FromFieldType> * col_from = typeid_cast<const ColumnConst<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
}
|
||||
|
||||
static void vector_constant(const PODArray<FromType> & vec_from, const std::string & data,
|
||||
PODArray<ToType> & vec_to)
|
||||
{
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
vec_to[i] = Op::execute(vec_from[i], remote_date_lut);
|
||||
}
|
||||
|
||||
static void vector_constant(const PODArray<FromType> & vec_from, PODArray<ToType> & vec_to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
vec_to[i] = Op::execute(vec_from[i], local_date_lut);
|
||||
}
|
||||
|
||||
static void constant_vector(const FromType & from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToType> & vec_to)
|
||||
{
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(), date_lut.toDayNum(col_from->getData()));
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Op::execute(from, remote_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
static void constant_constant(const FromType & from, const std::string & data, ToType & to)
|
||||
{
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
to = Op::execute(from, remote_date_lut);
|
||||
}
|
||||
|
||||
static void constant_constant(const FromType & from, ToType & to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
to = Op::execute(from, local_date_lut);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FromType, template <typename, typename> class Transformation, typename Name>
|
||||
class ToDateConverter
|
||||
{
|
||||
private:
|
||||
using FromFieldType = typename FromType::FieldType;
|
||||
using ToFieldType = typename DataTypeDate::FieldType;
|
||||
using Op = Transformer<FromFieldType, ToFieldType, Transformation>;
|
||||
|
||||
public:
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
const auto * sources = typeid_cast<const ColumnVector<FromFieldType> *>(&*source_col);
|
||||
const auto * const_source = typeid_cast<const ColumnConst<FromFieldType> *>(&*source_col);
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (sources)
|
||||
{
|
||||
auto * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const auto & vec_from = sources->getData();
|
||||
auto & vec_to = col_to->getData();
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
Op::vector_constant(vec_from, vec_to);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
ToFieldType res;
|
||||
Op::constant_constant(const_source->getData(), res);
|
||||
block.getByPosition(result).column = new ColumnConst<ToFieldType>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
||||
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
||||
|
||||
if (sources)
|
||||
{
|
||||
auto * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getData();
|
||||
auto & vec_to = col_to->getData();
|
||||
vec_to.resize(vec_from.size());
|
||||
|
||||
if (time_zones)
|
||||
Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
else if (const_time_zone)
|
||||
Op::vector_constant(vec_from, const_time_zone->getData(), vec_to);
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
if (time_zones)
|
||||
{
|
||||
auto * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_to = col_to->getData();
|
||||
vec_to.resize(time_zones->getOffsets().size());
|
||||
|
||||
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
}
|
||||
else if (const_time_zone)
|
||||
{
|
||||
ToFieldType res;
|
||||
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||
block.getByPosition(result).column = new ColumnConst<ToFieldType>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("FunctionsConversion: Internal error", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType>
|
||||
struct ToDateTransform
|
||||
{
|
||||
static inline ToType execute(const FromType & from, const DateLUTImpl & date_lut)
|
||||
{
|
||||
return date_lut.toDayNum(from);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType>
|
||||
struct ToDateTransform32Or64
|
||||
{
|
||||
static inline ToType execute(const FromType & from, const DateLUTImpl & date_lut)
|
||||
{
|
||||
return (from < 0xFFFF) ? from : date_lut.toDayNum(from);
|
||||
}
|
||||
};
|
||||
|
||||
}}
|
||||
|
||||
/** Преобразование даты-с-временем в дату: отбрасывание времени.
|
||||
*/
|
||||
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name> : details::ToDateConverter<DataTypeDateTime, details::ToDateTransform, Name> {};
|
||||
|
||||
/** Отдельный случай для преобразования (U)Int32 или (U)Int64 в Date.
|
||||
* Если число меньше 65536, то оно понимается, как DayNum, а если больше или равно - как unix timestamp.
|
||||
@ -142,56 +294,10 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
||||
* когда пользователь пишет toDate(UInt32), ожидая, что это - перевод unix timestamp в дату
|
||||
* (иначе такое использование было бы распространённой ошибкой).
|
||||
*/
|
||||
template <typename FromDataType, typename Name>
|
||||
struct ConvertImpl32Or64ToDate
|
||||
{
|
||||
typedef typename FromDataType::FieldType FromFieldType;
|
||||
typedef DataTypeDate::FieldType ToFieldType;
|
||||
|
||||
template <typename To, typename From>
|
||||
static To convert(const From & from, const DateLUTImpl & date_lut)
|
||||
{
|
||||
return from < 0xFFFF
|
||||
? from
|
||||
: date_lut.toDayNum(from);
|
||||
}
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from
|
||||
= typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnVector<ToFieldType> * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<FromFieldType>::Container_t & vec_from = col_from->getData();
|
||||
typename ColumnVector<ToFieldType>::Container_t & vec_to = col_to->getData();
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = convert<ToFieldType>(vec_from[i], date_lut);
|
||||
}
|
||||
else if (const ColumnConst<FromFieldType> * col_from
|
||||
= typeid_cast<const ColumnConst<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(),
|
||||
convert<ToFieldType>(col_from->getData(), date_lut));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name> : ConvertImpl32Or64ToDate<DataTypeUInt32, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name> : ConvertImpl32Or64ToDate<DataTypeUInt64, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate, Name> : ConvertImpl32Or64ToDate<DataTypeInt32, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate, Name> : ConvertImpl32Or64ToDate<DataTypeInt64, Name> {};
|
||||
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name> : details::ToDateConverter<DataTypeUInt32, details::ToDateTransform32Or64, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name> : details::ToDateConverter<DataTypeUInt64, details::ToDateTransform32Or64, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate, Name> : details::ToDateConverter<DataTypeInt32, details::ToDateTransform32Or64, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate, Name> : details::ToDateConverter<DataTypeInt64, details::ToDateTransform32Or64, Name> {};
|
||||
|
||||
/** Преобразование чисел, дат, дат-с-временем в строки: через форматирование.
|
||||
*/
|
||||
@ -887,6 +993,8 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
}
|
||||
};
|
||||
|
||||
/// Предварительное объявление.
|
||||
struct NameToDate { static constexpr auto name = "toDate"; };
|
||||
|
||||
template <typename ToDataType, typename Name>
|
||||
class FunctionConvert : public IFunction
|
||||
@ -934,7 +1042,9 @@ public:
|
||||
private:
|
||||
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||
typename std::enable_if<!(std::is_same<ToDataType2, DataTypeString>::value || std::is_same<Name2, NameToUnixTimestamp>::value), void>::type * = nullptr) const
|
||||
typename std::enable_if<!(std::is_same<ToDataType2, DataTypeString>::value ||
|
||||
std::is_same<Name2, NameToUnixTimestamp>::value ||
|
||||
std::is_same<Name2, NameToDate>::value)>::type * = nullptr) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
@ -960,7 +1070,7 @@ private:
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
else if ((arguments.size()) == 2 && typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr)
|
||||
else if ((arguments.size() == 2) && (typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||
@ -987,7 +1097,7 @@ private:
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
else if ((arguments.size()) == 2 && typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr)
|
||||
else if ((arguments.size() == 2) && (typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||
@ -997,6 +1107,26 @@ private:
|
||||
|
||||
return new ToDataType2;
|
||||
}
|
||||
|
||||
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||
typename std::enable_if<std::is_same<Name2, NameToDate>::value>::type * = nullptr) const
|
||||
{
|
||||
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 or 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if ((arguments.size() == 2) && (typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of 2nd argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
return new ToDataType2;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -1140,7 +1270,6 @@ struct NameToInt32 { static constexpr auto name = "toInt32"; };
|
||||
struct NameToInt64 { static constexpr auto name = "toInt64"; };
|
||||
struct NameToFloat32 { static constexpr auto name = "toFloat32"; };
|
||||
struct NameToFloat64 { static constexpr auto name = "toFloat64"; };
|
||||
struct NameToDate { static constexpr auto name = "toDate"; };
|
||||
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
|
||||
struct NameToString { static constexpr auto name = "toString"; };
|
||||
|
||||
|
@ -402,49 +402,8 @@ public:
|
||||
return name;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
return getReturnTypeImpl(arguments);
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
IDataType * from_type = &*block.getByPosition(arguments[0]).type;
|
||||
|
||||
if (typeid_cast<const DataTypeDate *>(from_type))
|
||||
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform, Name>::execute(block, arguments, result);
|
||||
else if (typeid_cast<const DataTypeDateTime * >(from_type))
|
||||
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform, 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);
|
||||
}
|
||||
|
||||
private:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
|
||||
template<typename ToDataType2 = ToDataType, typename Transform2 = Transform>
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||
typename std::enable_if<
|
||||
!(std::is_same<ToDataType2, DataTypeDate>::value
|
||||
|| (std::is_same<ToDataType2, DataTypeDateTime>::value && std::is_same<Transform2, ToTimeImpl>::value))
|
||||
, void>::type * = nullptr) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return new ToDataType;
|
||||
}
|
||||
|
||||
template<typename ToDataType2 = ToDataType, typename Transform2 = Transform>
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||
typename std::enable_if<
|
||||
std::is_same<ToDataType2, DataTypeDate>::value
|
||||
|| (std::is_same<ToDataType2, DataTypeDateTime>::value && std::is_same<Transform2, ToTimeImpl>::value)
|
||||
, void>::type * = nullptr) const
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if ((arguments.size() < 1) || (arguments.size() > 2))
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
@ -468,6 +427,20 @@ private:
|
||||
|
||||
return new ToDataType;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
IDataType * from_type = &*block.getByPosition(arguments[0]).type;
|
||||
|
||||
if (typeid_cast<const DataTypeDate *>(from_type))
|
||||
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform, Name>::execute(block, arguments, result);
|
||||
else if (typeid_cast<const DataTypeDateTime * >(from_type))
|
||||
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform, 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);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Functions/FunctionsArray.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <ext/range.hpp>
|
||||
|
||||
@ -44,7 +45,7 @@ namespace DB
|
||||
* Функции работы с URL расположены отдельно.
|
||||
* Функции кодирования строк, конвертации в другие типы расположены отдельно.
|
||||
*
|
||||
* Функции length, empty, notEmpty также работают с массивами.
|
||||
* Функции length, empty, notEmpty, reverse также работают с массивами.
|
||||
*/
|
||||
|
||||
|
||||
@ -921,11 +922,78 @@ public:
|
||||
};
|
||||
|
||||
|
||||
class FunctionConcat : public IFunction
|
||||
/// Также работает над массивами.
|
||||
class FunctionReverse : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "concat";
|
||||
static IFunction * create(const Context & context) { return new FunctionConcat; }
|
||||
static constexpr auto name = "reverse";
|
||||
static IFunction * create(const Context & context) { return new FunctionReverse; }
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!typeid_cast<const DataTypeString *>(&*arguments[0]) && !typeid_cast<const DataTypeFixedString *>(&*arguments[0])
|
||||
&& !typeid_cast<const DataTypeArray *>(&*arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0]->clone();
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
if (const ColumnString * col = typeid_cast<const ColumnString *>(column.get()))
|
||||
{
|
||||
ColumnString * col_res = new ColumnString;
|
||||
block.getByPosition(result).column = col_res;
|
||||
ReverseImpl::vector(col->getChars(), col->getOffsets(),
|
||||
col_res->getChars(), col_res->getOffsets());
|
||||
}
|
||||
else if (const ColumnFixedString * col = typeid_cast<const ColumnFixedString *>(column.get()))
|
||||
{
|
||||
ColumnFixedString * col_res = new ColumnFixedString(col->getN());
|
||||
block.getByPosition(result).column = col_res;
|
||||
ReverseImpl::vector_fixed(col->getChars(), col->getN(),
|
||||
col_res->getChars());
|
||||
}
|
||||
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(column.get()))
|
||||
{
|
||||
String res;
|
||||
ReverseImpl::constant(col->getData(), res);
|
||||
ColumnConstString * col_res = new ColumnConstString(col->size(), res);
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else if (typeid_cast<const ColumnArray *>(column.get()) || typeid_cast<const ColumnConstArray *>(column.get()))
|
||||
{
|
||||
FunctionArrayReverse().execute(block, arguments, result);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename Name>
|
||||
class ConcatImpl : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static IFunction * create(const Context & context) { return new ConcatImpl; }
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const override
|
||||
@ -1593,10 +1661,11 @@ struct NameLower { static constexpr auto name = "lower"; };
|
||||
struct NameUpper { static constexpr auto name = "upper"; };
|
||||
struct NameLowerUTF8 { static constexpr auto name = "lowerUTF8"; };
|
||||
struct NameUpperUTF8 { static constexpr auto name = "upperUTF8"; };
|
||||
struct NameReverse { static constexpr auto name = "reverse"; };
|
||||
struct NameReverseUTF8 { static constexpr auto name = "reverseUTF8"; };
|
||||
struct NameSubstring { static constexpr auto name = "substring"; };
|
||||
struct NameSubstringUTF8 { static constexpr auto name = "substringUTF8"; };
|
||||
struct NameConcat { static constexpr auto name = "concat"; };
|
||||
struct NameConcatAssumeInjective { static constexpr auto name = "concatAssumeInjective"; };
|
||||
|
||||
typedef FunctionStringOrArrayToT<EmptyImpl<false>, NameEmpty, UInt8> FunctionEmpty;
|
||||
typedef FunctionStringOrArrayToT<EmptyImpl<true>, NameNotEmpty, UInt8> FunctionNotEmpty;
|
||||
@ -1610,10 +1679,11 @@ typedef FunctionStringToString<
|
||||
typedef FunctionStringToString<
|
||||
LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>,
|
||||
NameUpperUTF8> FunctionUpperUTF8;
|
||||
typedef FunctionStringToString<ReverseImpl, NameReverse> FunctionReverse;
|
||||
typedef FunctionStringToString<ReverseUTF8Impl, NameReverseUTF8> FunctionReverseUTF8;
|
||||
typedef FunctionStringNumNumToString<SubstringImpl, NameSubstring> FunctionSubstring;
|
||||
typedef FunctionStringNumNumToString<SubstringUTF8Impl, NameSubstringUTF8> FunctionSubstringUTF8;
|
||||
using FunctionConcat = ConcatImpl<NameConcat>;
|
||||
using FunctionConcatAssumeInjective = ConcatImpl<NameConcatAssumeInjective>;
|
||||
|
||||
|
||||
}
|
||||
|
@ -23,8 +23,9 @@ namespace DB
|
||||
* - нулевой subpattern (сматчившуюся часть, иначе);
|
||||
* - инача, пустой массив
|
||||
*
|
||||
* join(sep, arr)
|
||||
* join(arr)
|
||||
* arrayStringConcat(arr)
|
||||
* arrayStringConcat(arr, delimiter)
|
||||
* - склеить массив строк в одну строку через разделитель.
|
||||
*
|
||||
* alphaTokens(s) - выделить из строки подпоследовательности [a-zA-Z]+.
|
||||
*
|
||||
@ -414,8 +415,142 @@ public:
|
||||
};
|
||||
|
||||
|
||||
typedef FunctionTokens<AlphaTokensImpl> FunctionAlphaTokens;
|
||||
typedef FunctionTokens<SplitByCharImpl> FunctionSplitByChar;
|
||||
/// Склеивает массив строк в одну строку через разделитель.
|
||||
class FunctionArrayStringConcat : public IFunction
|
||||
{
|
||||
private:
|
||||
void executeImpl(
|
||||
const ColumnString::Chars_t & src_chars,
|
||||
const ColumnString::Offsets_t & src_string_offsets,
|
||||
const ColumnArray::Offsets_t & src_array_offsets,
|
||||
const char * delimiter, const size_t delimiter_size,
|
||||
ColumnString::Chars_t & dst_chars,
|
||||
ColumnString::Offsets_t & dst_string_offsets)
|
||||
{
|
||||
size_t size = src_array_offsets.size();
|
||||
|
||||
if (!size)
|
||||
return;
|
||||
|
||||
/// С небольшим запасом - как будто разделитель идёт и после последней строки массива.
|
||||
dst_chars.resize(
|
||||
src_chars.size()
|
||||
+ delimiter_size * src_string_offsets.size() /// Разделители после каждой строки...
|
||||
+ src_array_offsets.size() /// Нулевой байт после каждой склеенной строки
|
||||
- src_string_offsets.size()); /// Бывший нулевой байт после каждой строки массива
|
||||
|
||||
/// Будет столько строк, сколько было массивов.
|
||||
dst_string_offsets.resize(src_array_offsets.size());
|
||||
|
||||
ColumnArray::Offset_t current_src_array_offset = 0;
|
||||
ColumnString::Offset_t current_src_string_offset = 0;
|
||||
|
||||
ColumnString::Offset_t current_dst_string_offset = 0;
|
||||
|
||||
/// Цикл по массивам строк.
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
/// Цикл по строкам внутри массива. /// NOTE Можно всё сделать за одно копирование, если разделитель имеет размер 1.
|
||||
for (auto next_src_array_offset = src_array_offsets[i]; current_src_array_offset < next_src_array_offset; ++current_src_array_offset)
|
||||
{
|
||||
size_t bytes_to_copy = src_string_offsets[current_src_array_offset] - current_src_string_offset - 1;
|
||||
|
||||
memcpy(&dst_chars[current_dst_string_offset], &src_chars[current_src_string_offset], bytes_to_copy);
|
||||
|
||||
current_src_string_offset = src_string_offsets[current_src_array_offset];
|
||||
current_dst_string_offset += bytes_to_copy;
|
||||
|
||||
if (current_src_array_offset + 1 != next_src_array_offset)
|
||||
{
|
||||
memcpy(&dst_chars[current_dst_string_offset], delimiter, delimiter_size);
|
||||
current_dst_string_offset += delimiter_size;
|
||||
}
|
||||
}
|
||||
|
||||
dst_chars[current_dst_string_offset] = 0;
|
||||
++current_dst_string_offset;
|
||||
|
||||
dst_string_offsets[i] = current_dst_string_offset;
|
||||
}
|
||||
|
||||
dst_chars.resize(dst_string_offsets.back());
|
||||
}
|
||||
|
||||
public:
|
||||
static constexpr auto name = "arrayStringConcat";
|
||||
static IFunction * create(const Context & context) { return new FunctionArrayStringConcat; }
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
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 or 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type || !typeid_cast<const DataTypeString *>(array_type->getNestedType().get()))
|
||||
throw Exception("First argument for function " + getName() + " must be array of strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 2
|
||||
&& !typeid_cast<const DataTypeString *>(arguments[1].get()))
|
||||
throw Exception("Second argument for function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
String delimiter;
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnConstString * col_delim = typeid_cast<const ColumnConstString *>(block.getByPosition(arguments[1]).column.get());
|
||||
if (!col_delim)
|
||||
throw Exception("Second argument for function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
delimiter = col_delim->getData();
|
||||
}
|
||||
|
||||
if (const ColumnConstArray * col_const_arr = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
ColumnConstString * col_res = new ColumnConstString(col_const_arr->size(), "");
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
const Array & src_arr = col_const_arr->getData();
|
||||
String & dst_str = col_res->getData();
|
||||
for (size_t i = 0, size = src_arr.size(); i < size; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
dst_str += delimiter;
|
||||
dst_str += src_arr[i].get<const String &>();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
const ColumnArray & col_arr = static_cast<const ColumnArray &>(*block.getByPosition(arguments[0]).column);
|
||||
const ColumnString & col_string = static_cast<const ColumnString &>(col_arr.getData());
|
||||
|
||||
ColumnString * col_res = new ColumnString;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
executeImpl(
|
||||
col_string.getChars(), col_string.getOffsets(), col_arr.getOffsets(),
|
||||
delimiter.data(), delimiter.size(),
|
||||
col_res->getChars(), col_res->getOffsets());
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
typedef FunctionTokens<AlphaTokensImpl> FunctionAlphaTokens;
|
||||
typedef FunctionTokens<SplitByCharImpl> FunctionSplitByChar;
|
||||
typedef FunctionTokens<SplitByStringImpl> FunctionSplitByString;
|
||||
typedef FunctionTokens<ExtractAllImpl> FunctionExtractAll;
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Common/Volnitsky.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <DB/Common/StringSearcher.h>
|
||||
#include <re2/re2.h>
|
||||
#include <re2/stringpiece.h>
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
@ -52,6 +53,7 @@ namespace DB
|
||||
*/
|
||||
|
||||
|
||||
template <bool CaseSensitive>
|
||||
struct PositionImpl
|
||||
{
|
||||
typedef UInt64 ResultType;
|
||||
@ -69,7 +71,7 @@ struct PositionImpl
|
||||
/// Текущий индекс в массиве строк.
|
||||
size_t i = 0;
|
||||
|
||||
Volnitsky searcher(needle.data(), needle.size(), end - pos);
|
||||
VolnitskyImpl<CaseSensitive, true> searcher(needle.data(), needle.size(), end - pos);
|
||||
|
||||
/// Искать будем следующее вхождение сразу во всех строках.
|
||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||
@ -94,8 +96,14 @@ struct PositionImpl
|
||||
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
|
||||
static void constant(const std::string & data, const std::string & needle, UInt64 & res)
|
||||
static void constant(std::string data, std::string needle, UInt64 & res)
|
||||
{
|
||||
if (!CaseSensitive)
|
||||
{
|
||||
std::transform(std::begin(data), std::end(data), std::begin(data), tolower);
|
||||
std::transform(std::begin(needle), std::end(needle), std::begin(needle), tolower);
|
||||
}
|
||||
|
||||
res = data.find(needle);
|
||||
if (res == std::string::npos)
|
||||
res = 0;
|
||||
@ -105,43 +113,7 @@ struct PositionImpl
|
||||
};
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
|
||||
const UInt8 utf8_continuation_octet_mask = 0b11000000u;
|
||||
const UInt8 utf8_continuation_octet = 0b10000000u;
|
||||
|
||||
|
||||
/// return true if `octet` binary repr starts with 10 (octet is a UTF-8 sequence continuation)
|
||||
bool utf8_is_continuation_octet(const UInt8 octet)
|
||||
{
|
||||
return (octet & utf8_continuation_octet_mask) == utf8_continuation_octet;
|
||||
}
|
||||
|
||||
/// moves `s` forward until either first non-continuation octet or string end is met
|
||||
void utf8_sync_forward(const UInt8 * & s, const UInt8 * const end = nullptr)
|
||||
{
|
||||
while (s < end && utf8_is_continuation_octet(*s))
|
||||
++s;
|
||||
}
|
||||
|
||||
/// returns UTF-8 code point sequence length judging by it's first octet
|
||||
std::size_t utf8_seq_length(const UInt8 first_octet)
|
||||
{
|
||||
if (first_octet < 0x80u)
|
||||
return 1;
|
||||
|
||||
const std::size_t bits = 8;
|
||||
const auto first_zero = _bit_scan_reverse(static_cast<UInt8>(~first_octet));
|
||||
|
||||
return bits - 1 - first_zero;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
template <bool CaseSensitive>
|
||||
struct PositionUTF8Impl
|
||||
{
|
||||
typedef UInt64 ResultType;
|
||||
@ -157,7 +129,7 @@ struct PositionUTF8Impl
|
||||
/// Текущий индекс в массиве строк.
|
||||
size_t i = 0;
|
||||
|
||||
Volnitsky searcher(needle.data(), needle.size(), end - pos);
|
||||
VolnitskyImpl<CaseSensitive, false> searcher(needle.data(), needle.size(), end - pos);
|
||||
|
||||
/// Искать будем следующее вхождение сразу во всех строках.
|
||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||
@ -175,7 +147,7 @@ struct PositionUTF8Impl
|
||||
/// А теперь надо найти, сколько кодовых точек находится перед pos.
|
||||
res[i] = 1;
|
||||
for (const UInt8 * c = begin + (i != 0 ? offsets[i - 1] : 0); c < pos; ++c)
|
||||
if (!utf8_is_continuation_octet(*c))
|
||||
if (!UTF8::isContinuationOctet(*c))
|
||||
++res[i];
|
||||
}
|
||||
else
|
||||
@ -188,15 +160,36 @@ struct PositionUTF8Impl
|
||||
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
|
||||
static void constant(const std::string & data, const std::string & needle, UInt64 & res)
|
||||
static void constant(std::string data, std::string needle, UInt64 & res)
|
||||
{
|
||||
if (!CaseSensitive)
|
||||
{
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
|
||||
auto data_pos = reinterpret_cast<UInt8 *>(&data[0]);
|
||||
const auto data_end = data_pos + data.size();
|
||||
while (data_pos < data_end)
|
||||
{
|
||||
const auto len = utf8.convert(Poco::Unicode::toLower(utf8.convert(data_pos)), data_pos, data_end - data_pos);
|
||||
data_pos += len;
|
||||
}
|
||||
|
||||
auto needle_pos = reinterpret_cast<UInt8 *>(&needle[0]);
|
||||
const auto needle_end = needle_pos + needle.size();
|
||||
while (needle_pos < needle_end)
|
||||
{
|
||||
const auto len = utf8.convert(Poco::Unicode::toLower(utf8.convert(needle_pos)), needle_pos, needle_end - needle_pos);
|
||||
needle_pos += len;
|
||||
}
|
||||
}
|
||||
|
||||
const auto pos = data.find(needle);
|
||||
if (pos != std::string::npos)
|
||||
{
|
||||
/// А теперь надо найти, сколько кодовых точек находится перед pos.
|
||||
res = 1;
|
||||
for (const auto i : ext::range(0, pos))
|
||||
if (!utf8_is_continuation_octet(static_cast<UInt8>(data[i])))
|
||||
if (!UTF8::isContinuationOctet(static_cast<UInt8>(data[i])))
|
||||
++res;
|
||||
}
|
||||
else
|
||||
@ -207,143 +200,6 @@ struct PositionUTF8Impl
|
||||
|
||||
struct PositionCaseInsensitiveImpl
|
||||
{
|
||||
private:
|
||||
class CaseInsensitiveSearcher
|
||||
{
|
||||
static constexpr auto n = sizeof(__m128i);
|
||||
|
||||
const int page_size = getpagesize();
|
||||
|
||||
/// string to be searched for
|
||||
const std::string & needle;
|
||||
/// lower and uppercase variants of the first character in `needle`
|
||||
UInt8 l{};
|
||||
UInt8 u{};
|
||||
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
|
||||
__m128i patl, patu;
|
||||
/// lower and uppercase vectors of first 16 characters of `needle`
|
||||
__m128i cachel = _mm_setzero_si128(), cacheu = _mm_setzero_si128();
|
||||
int cachemask{};
|
||||
|
||||
bool page_safe(const void * const ptr) const
|
||||
{
|
||||
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
|
||||
}
|
||||
|
||||
public:
|
||||
CaseInsensitiveSearcher(const std::string & needle) : needle(needle)
|
||||
{
|
||||
if (needle.empty())
|
||||
return;
|
||||
|
||||
auto needle_pos = needle.data();
|
||||
|
||||
l = std::tolower(*needle_pos);
|
||||
u = std::toupper(*needle_pos);
|
||||
|
||||
patl = _mm_set1_epi8(l);
|
||||
patu = _mm_set1_epi8(u);
|
||||
|
||||
const auto needle_end = needle_pos + needle.size();
|
||||
|
||||
for (const auto i : ext::range(0, n))
|
||||
{
|
||||
cachel = _mm_srli_si128(cachel, 1);
|
||||
cacheu = _mm_srli_si128(cacheu, 1);
|
||||
|
||||
if (needle_pos != needle_end)
|
||||
{
|
||||
cachel = _mm_insert_epi8(cachel, std::tolower(*needle_pos), n - 1);
|
||||
cacheu = _mm_insert_epi8(cacheu, std::toupper(*needle_pos), n - 1);
|
||||
cachemask |= 1 << i;
|
||||
++needle_pos;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
const UInt8 * find(const UInt8 * haystack, const UInt8 * const haystack_end) const
|
||||
{
|
||||
if (needle.empty())
|
||||
return haystack;
|
||||
|
||||
const auto needle_begin = reinterpret_cast<const UInt8 *>(needle.data());
|
||||
const auto needle_end = needle_begin + needle.size();
|
||||
|
||||
while (haystack < haystack_end)
|
||||
{
|
||||
/// @todo supposedly for long strings spanning across multiple pages. Why don't we use this technique in other places?
|
||||
if (haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, patl);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, patu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (mask == 0)
|
||||
{
|
||||
haystack += n;
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto offset = _bit_scan_forward(mask);
|
||||
haystack += offset;
|
||||
|
||||
if (haystack < haystack_end && haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (0xffff == cachemask)
|
||||
{
|
||||
if (mask == cachemask)
|
||||
{
|
||||
auto haystack_pos = haystack + n;
|
||||
auto needle_pos = needle_begin + n;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
|
||||
++haystack_pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
}
|
||||
else if ((mask & cachemask) == cachemask)
|
||||
return haystack;
|
||||
|
||||
++haystack;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
if (haystack == haystack_end)
|
||||
return haystack_end;
|
||||
|
||||
if (*haystack == l || *haystack == u)
|
||||
{
|
||||
auto haystack_pos = haystack + 1;
|
||||
auto needle_pos = needle_begin + 1;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
|
||||
++haystack_pos, ++needle_pos;
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
|
||||
++haystack;
|
||||
}
|
||||
|
||||
return haystack_end;
|
||||
}
|
||||
};
|
||||
|
||||
public:
|
||||
using ResultType = UInt64;
|
||||
|
||||
@ -351,7 +207,7 @@ public:
|
||||
const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, const std::string & needle,
|
||||
PODArray<UInt64> & res)
|
||||
{
|
||||
const CaseInsensitiveSearcher searcher{needle};
|
||||
const ASCIICaseInsensitiveStringSearcher searcher{needle.data(), needle.size()};
|
||||
|
||||
const UInt8 * begin = &data[0];
|
||||
const UInt8 * pos = begin;
|
||||
@ -361,7 +217,7 @@ public:
|
||||
size_t i = 0;
|
||||
|
||||
/// Искать будем следующее вхождение сразу во всех строках.
|
||||
while (pos < end && end != (pos = searcher.find(pos, end)))
|
||||
while (pos < end && end != (pos = searcher.search(pos, end)))
|
||||
{
|
||||
/// Определим, к какому индексу оно относится.
|
||||
while (begin + offsets[i] <= pos)
|
||||
@ -397,297 +253,6 @@ public:
|
||||
};
|
||||
|
||||
|
||||
struct PositionCaseInsensitiveUTF8Impl
|
||||
{
|
||||
private:
|
||||
class CaseInsensitiveSearcher
|
||||
{
|
||||
using UTF8SequenceBuffer = UInt8[6];
|
||||
|
||||
static constexpr auto n = sizeof(__m128i);
|
||||
|
||||
const int page_size = getpagesize();
|
||||
|
||||
/// string to be searched for
|
||||
const std::string & needle;
|
||||
bool first_needle_symbol_is_ascii{};
|
||||
/// lower and uppercase variants of the first octet of the first character in `needle`
|
||||
UInt8 l{};
|
||||
UInt8 u{};
|
||||
/// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
|
||||
__m128i patl, patu;
|
||||
/// lower and uppercase vectors of first 16 characters of `needle`
|
||||
__m128i cachel = _mm_setzero_si128(), cacheu = _mm_setzero_si128();
|
||||
int cachemask{};
|
||||
std::size_t cache_valid_len{};
|
||||
std::size_t cache_actual_len{};
|
||||
|
||||
bool page_safe(const void * const ptr) const
|
||||
{
|
||||
return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
|
||||
}
|
||||
|
||||
public:
|
||||
CaseInsensitiveSearcher(const std::string & needle) : needle(needle)
|
||||
{
|
||||
if (needle.empty())
|
||||
return;
|
||||
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
UTF8SequenceBuffer l_seq, u_seq;
|
||||
|
||||
auto needle_pos = reinterpret_cast<const UInt8 *>(needle.data());
|
||||
if (*needle_pos < 0x80u)
|
||||
{
|
||||
first_needle_symbol_is_ascii = true;
|
||||
l = std::tolower(*needle_pos);
|
||||
u = std::toupper(*needle_pos);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto first_u32 = utf8.convert(needle_pos);
|
||||
const auto first_l_u32 = Poco::Unicode::toLower(first_u32);
|
||||
const auto first_u_u32 = Poco::Unicode::toUpper(first_u32);
|
||||
|
||||
/// lower and uppercase variants of the first octet of the first character in `needle`
|
||||
utf8.convert(first_l_u32, l_seq, sizeof(l_seq));
|
||||
l = l_seq[0];
|
||||
utf8.convert(first_u_u32, u_seq, sizeof(u_seq));
|
||||
u = u_seq[0];
|
||||
}
|
||||
|
||||
/// for detecting leftmost position of the first symbol
|
||||
patl = _mm_set1_epi8(l);
|
||||
patu = _mm_set1_epi8(u);
|
||||
/// lower and uppercase vectors of first 16 octets of `needle`
|
||||
|
||||
const auto needle_end = needle_pos + needle.size();
|
||||
|
||||
for (std::size_t i = 0; i < n;)
|
||||
{
|
||||
if (needle_pos == needle_end)
|
||||
{
|
||||
cachel = _mm_srli_si128(cachel, 1);
|
||||
cacheu = _mm_srli_si128(cacheu, 1);
|
||||
++i;
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto src_len = utf8_seq_length(*needle_pos);
|
||||
const auto c_u32 = utf8.convert(needle_pos);
|
||||
|
||||
const auto c_l_u32 = Poco::Unicode::toLower(c_u32);
|
||||
const auto c_u_u32 = Poco::Unicode::toUpper(c_u32);
|
||||
|
||||
const auto dst_l_len = static_cast<UInt8>(utf8.convert(c_l_u32, l_seq, sizeof(l_seq)));
|
||||
const auto dst_u_len = static_cast<UInt8>(utf8.convert(c_u_u32, u_seq, sizeof(u_seq)));
|
||||
|
||||
/// @note Unicode standard states it is a rare but possible occasion
|
||||
if (!(dst_l_len == dst_u_len && dst_u_len == src_len))
|
||||
throw Exception{
|
||||
"UTF8 sequences with different lowercase and uppercase lengths are not supported",
|
||||
ErrorCodes::UNSUPPORTED_PARAMETER
|
||||
};
|
||||
|
||||
cache_actual_len += src_len;
|
||||
if (cache_actual_len < n)
|
||||
cache_valid_len += src_len;
|
||||
|
||||
for (std::size_t j = 0; j < src_len && i < n; ++j, ++i)
|
||||
{
|
||||
cachel = _mm_srli_si128(cachel, 1);
|
||||
cacheu = _mm_srli_si128(cacheu, 1);
|
||||
|
||||
if (needle_pos != needle_end)
|
||||
{
|
||||
cachel = _mm_insert_epi8(cachel, l_seq[j], n - 1);
|
||||
cacheu = _mm_insert_epi8(cacheu, u_seq[j], n - 1);
|
||||
|
||||
cachemask |= 1 << i;
|
||||
++needle_pos;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
const UInt8 * find(const UInt8 * haystack, const UInt8 * const haystack_end) const
|
||||
{
|
||||
if (needle.empty())
|
||||
return haystack;
|
||||
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
|
||||
const auto needle_begin = reinterpret_cast<const UInt8 *>(needle.data());
|
||||
const auto needle_end = needle_begin + needle.size();
|
||||
|
||||
while (haystack < haystack_end)
|
||||
{
|
||||
if (haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, patl);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, patu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (mask == 0)
|
||||
{
|
||||
haystack += n;
|
||||
utf8_sync_forward(haystack, haystack_end);
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto offset = _bit_scan_forward(mask);
|
||||
haystack += offset;
|
||||
|
||||
if (haystack < haystack_end && haystack + n <= haystack_end && page_safe(haystack))
|
||||
{
|
||||
const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
|
||||
const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
|
||||
const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
|
||||
const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
|
||||
const auto mask = _mm_movemask_epi8(v_against_l_or_u);
|
||||
|
||||
if (0xffff == cachemask)
|
||||
{
|
||||
if (mask == cachemask)
|
||||
{
|
||||
auto haystack_pos = haystack + cache_valid_len;
|
||||
auto needle_pos = needle_begin + cache_valid_len;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
Poco::Unicode::toLower(utf8.convert(haystack_pos)) ==
|
||||
Poco::Unicode::toLower(utf8.convert(needle_pos)))
|
||||
{
|
||||
/// @note assuming sequences for lowercase and uppercase have exact same length
|
||||
const auto len = utf8_seq_length(*haystack_pos);
|
||||
haystack_pos += len, needle_pos += len;
|
||||
}
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
}
|
||||
else if ((mask & cachemask) == cachemask)
|
||||
return haystack;
|
||||
|
||||
/// first octet was ok, but not the first 16, move to start of next sequence and reapply
|
||||
haystack += utf8_seq_length(*haystack);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
if (haystack == haystack_end)
|
||||
return haystack_end;
|
||||
|
||||
if (*haystack == l || *haystack == u)
|
||||
{
|
||||
auto haystack_pos = haystack + first_needle_symbol_is_ascii;
|
||||
auto needle_pos = needle_begin + first_needle_symbol_is_ascii;
|
||||
|
||||
while (haystack_pos < haystack_end && needle_pos < needle_end &&
|
||||
Poco::Unicode::toLower(utf8.convert(haystack_pos)) ==
|
||||
Poco::Unicode::toLower(utf8.convert(needle_pos)))
|
||||
{
|
||||
const auto len = utf8_seq_length(*haystack_pos);
|
||||
haystack_pos += len, needle_pos += len;
|
||||
}
|
||||
|
||||
if (needle_pos == needle_end)
|
||||
return haystack;
|
||||
}
|
||||
|
||||
/// advance to the start of the next sequence
|
||||
haystack += utf8_seq_length(*haystack);
|
||||
}
|
||||
|
||||
return haystack_end;
|
||||
}
|
||||
};
|
||||
|
||||
public:
|
||||
using ResultType = UInt64;
|
||||
|
||||
static void vector(
|
||||
const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, const std::string & needle,
|
||||
PODArray<UInt64> & res)
|
||||
{
|
||||
const CaseInsensitiveSearcher searcher{needle};
|
||||
|
||||
const UInt8 * begin = &data[0];
|
||||
const UInt8 * pos = begin;
|
||||
const UInt8 * end = pos + data.size();
|
||||
|
||||
/// Текущий индекс в массиве строк.
|
||||
size_t i = 0;
|
||||
|
||||
/// Искать будем следующее вхождение сразу во всех строках.
|
||||
while (pos < end && end != (pos = searcher.find(pos, end)))
|
||||
{
|
||||
/// Определим, к какому индексу оно относится.
|
||||
while (begin + offsets[i] <= pos)
|
||||
{
|
||||
res[i] = 0;
|
||||
++i;
|
||||
}
|
||||
|
||||
/// Проверяем, что вхождение не переходит через границы строк.
|
||||
if (pos + needle.size() < begin + offsets[i])
|
||||
{
|
||||
/// А теперь надо найти, сколько кодовых точек находится перед pos.
|
||||
res[i] = 1;
|
||||
for (const UInt8 * c = begin + (i != 0 ? offsets[i - 1] : 0); c < pos; ++c)
|
||||
if (!utf8_is_continuation_octet(*c))
|
||||
++res[i];
|
||||
}
|
||||
else
|
||||
res[i] = 0;
|
||||
|
||||
pos = begin + offsets[i];
|
||||
++i;
|
||||
}
|
||||
|
||||
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
|
||||
static void constant(std::string data, std::string needle, UInt64 & res)
|
||||
{
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
|
||||
auto data_pos = reinterpret_cast<UInt8 *>(&data[0]);
|
||||
const auto data_end = data_pos + data.size();
|
||||
while (data_pos < data_end)
|
||||
{
|
||||
const auto len = utf8.convert(Poco::Unicode::toLower(utf8.convert(data_pos)), data_pos, data_end - data_pos);
|
||||
data_pos += len;
|
||||
}
|
||||
|
||||
auto needle_pos = reinterpret_cast<UInt8 *>(&needle[0]);
|
||||
const auto needle_end = needle_pos + needle.size();
|
||||
while (needle_pos < needle_end)
|
||||
{
|
||||
const auto len = utf8.convert(Poco::Unicode::toLower(utf8.convert(needle_pos)), needle_pos, needle_end - needle_pos);
|
||||
needle_pos += len;
|
||||
}
|
||||
|
||||
const auto pos = data.find(needle);
|
||||
if (pos != std::string::npos)
|
||||
{
|
||||
/// А теперь надо найти, сколько кодовых точек находится перед pos.
|
||||
res = 1;
|
||||
for (const auto i : ext::range(0, pos))
|
||||
if (!utf8_is_continuation_octet(static_cast<UInt8>(data[i])))
|
||||
++res;
|
||||
}
|
||||
else
|
||||
res = 0;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
||||
/// Переводит выражение LIKE в regexp re2. Например, abc%def -> ^abc.*def$
|
||||
inline String likePatternToRegexp(const String & pattern)
|
||||
{
|
||||
@ -1744,10 +1309,11 @@ struct NameReplaceAll { static constexpr auto name = "replaceAll"; };
|
||||
struct NameReplaceRegexpOne { static constexpr auto name = "replaceRegexpOne"; };
|
||||
struct NameReplaceRegexpAll { static constexpr auto name = "replaceRegexpAll"; };
|
||||
|
||||
typedef FunctionsStringSearch<PositionImpl, NamePosition> FunctionPosition;
|
||||
typedef FunctionsStringSearch<PositionUTF8Impl, NamePositionUTF8> FunctionPositionUTF8;
|
||||
typedef FunctionsStringSearch<PositionImpl<true>, NamePosition> FunctionPosition;
|
||||
typedef FunctionsStringSearch<PositionUTF8Impl<true>, NamePositionUTF8> FunctionPositionUTF8;
|
||||
typedef FunctionsStringSearch<PositionCaseInsensitiveImpl, NamePositionCaseInsensitive> FunctionPositionCaseInsensitive;
|
||||
typedef FunctionsStringSearch<PositionCaseInsensitiveUTF8Impl, NamePositionCaseInsensitiveUTF8> FunctionPositionCaseInsensitiveUTF8;
|
||||
typedef FunctionsStringSearch<PositionUTF8Impl<false>, NamePositionCaseInsensitiveUTF8> FunctionPositionCaseInsensitiveUTF8;
|
||||
|
||||
typedef FunctionsStringSearch<MatchImpl<false>, NameMatch> FunctionMatch;
|
||||
typedef FunctionsStringSearch<MatchImpl<true>, NameLike> FunctionLike;
|
||||
typedef FunctionsStringSearch<MatchImpl<true, true>, NameNotLike> FunctionNotLike;
|
||||
|
@ -112,6 +112,11 @@ inline void assertString(const String & s, ReadBuffer & buf)
|
||||
assertString(s.c_str(), buf);
|
||||
}
|
||||
|
||||
bool checkString(const char * s, ReadBuffer & buf);
|
||||
inline bool checkString(const String & s, ReadBuffer & buf)
|
||||
{
|
||||
return checkString(s.c_str(), buf);
|
||||
}
|
||||
|
||||
inline void readBoolText(bool & x, ReadBuffer & buf)
|
||||
{
|
||||
@ -120,14 +125,20 @@ inline void readBoolText(bool & x, ReadBuffer & buf)
|
||||
x = tmp != '0';
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
void readIntText(T & x, ReadBuffer & buf)
|
||||
template <typename T, typename ReturnType = void>
|
||||
ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
|
||||
{
|
||||
static constexpr bool throw_exception = std::is_same<ReturnType, void>::value;
|
||||
|
||||
bool negative = false;
|
||||
x = 0;
|
||||
if (buf.eof())
|
||||
throwReadAfterEOF();
|
||||
{
|
||||
if (throw_exception)
|
||||
throwReadAfterEOF();
|
||||
else
|
||||
return ReturnType(false);
|
||||
}
|
||||
|
||||
while (!buf.eof())
|
||||
{
|
||||
@ -139,7 +150,7 @@ void readIntText(T & x, ReadBuffer & buf)
|
||||
if (std::is_signed<T>::value)
|
||||
negative = true;
|
||||
else
|
||||
return;
|
||||
return ReturnType(false);
|
||||
break;
|
||||
case '0':
|
||||
case '1':
|
||||
@ -157,14 +168,27 @@ void readIntText(T & x, ReadBuffer & buf)
|
||||
default:
|
||||
if (negative)
|
||||
x = -x;
|
||||
return;
|
||||
return ReturnType(true);
|
||||
}
|
||||
++buf.position();
|
||||
}
|
||||
if (negative)
|
||||
x = -x;
|
||||
|
||||
return ReturnType(true);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void readIntText(T & x, ReadBuffer & buf)
|
||||
{
|
||||
readIntTextImpl<T, void>(x, buf);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool tryReadIntText(T & x, ReadBuffer & buf)
|
||||
{
|
||||
return readIntTextImpl<T, bool>(x, buf);
|
||||
}
|
||||
|
||||
/** Более оптимизированная версия (примерно в 1.5 раза на реальных данных).
|
||||
* Отличается тем, что:
|
||||
@ -211,18 +235,53 @@ void readIntTextUnsafe(T & x, ReadBuffer & buf)
|
||||
x = -x;
|
||||
}
|
||||
|
||||
template <bool throw_exception, class ExcepFun, class NoExcepFun, class... Args>
|
||||
bool exceptionPolicySelector(ExcepFun && excep_f, NoExcepFun && no_excep_f, Args &&... args)
|
||||
{
|
||||
if (throw_exception)
|
||||
{
|
||||
excep_f(std::forward<Args>(args)...);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return no_excep_f(std::forward<Args>(args)...);
|
||||
};
|
||||
|
||||
|
||||
/// грубо
|
||||
template <typename T>
|
||||
void readFloatText(T & x, ReadBuffer & buf)
|
||||
template <typename T, typename ReturnType>
|
||||
ReturnType readFloatTextImpl(T & x, ReadBuffer & buf)
|
||||
{
|
||||
static constexpr bool throw_exception = std::is_same<ReturnType, void>::value;
|
||||
|
||||
bool negative = false;
|
||||
x = 0;
|
||||
bool after_point = false;
|
||||
double power_of_ten = 1;
|
||||
|
||||
if (buf.eof())
|
||||
throwReadAfterEOF();
|
||||
{
|
||||
if (throw_exception)
|
||||
throwReadAfterEOF();
|
||||
else
|
||||
return ReturnType(false);
|
||||
}
|
||||
|
||||
auto parse_special_value = [&buf, &x, &negative](const char * str, T value)
|
||||
{
|
||||
auto assert_str_lambda = [](const char * str, ReadBuffer & buf){ assertString(str, buf); };
|
||||
auto check_str_lambda = [](const char * str, ReadBuffer & buf){ return checkString(str, buf); };
|
||||
|
||||
++buf.position();
|
||||
bool result = exceptionPolicySelector<throw_exception>(assert_str_lambda, check_str_lambda, str, buf);
|
||||
if (result)
|
||||
{
|
||||
x = value;
|
||||
if (negative)
|
||||
x = -x;
|
||||
}
|
||||
return result;
|
||||
};
|
||||
|
||||
while (!buf.eof())
|
||||
{
|
||||
@ -262,47 +321,51 @@ void readFloatText(T & x, ReadBuffer & buf)
|
||||
{
|
||||
++buf.position();
|
||||
Int32 exponent = 0;
|
||||
readIntText(exponent, buf);
|
||||
x *= exp10(exponent);
|
||||
if (negative)
|
||||
x = -x;
|
||||
return;
|
||||
bool res = exceptionPolicySelector<throw_exception>(readIntText<Int32>, tryReadIntText<Int32>, exponent, buf);
|
||||
if (res)
|
||||
{
|
||||
x *= exp10(exponent);
|
||||
if (negative)
|
||||
x = -x;
|
||||
}
|
||||
return ReturnType(res);
|
||||
}
|
||||
case 'i':
|
||||
++buf.position();
|
||||
assertString("nf", buf);
|
||||
x = std::numeric_limits<T>::infinity();
|
||||
if (negative)
|
||||
x = -x;
|
||||
return;
|
||||
return ReturnType(parse_special_value("nf", std::numeric_limits<T>::infinity()));
|
||||
|
||||
case 'I':
|
||||
++buf.position();
|
||||
assertString("NF", buf);
|
||||
x = std::numeric_limits<T>::infinity();
|
||||
if (negative)
|
||||
x = -x;
|
||||
return;
|
||||
return ReturnType(parse_special_value("NF", std::numeric_limits<T>::infinity()));
|
||||
|
||||
case 'n':
|
||||
++buf.position();
|
||||
assertString("an", buf);
|
||||
x = std::numeric_limits<T>::quiet_NaN();
|
||||
return;
|
||||
return ReturnType(parse_special_value("an", std::numeric_limits<T>::quiet_NaN()));
|
||||
|
||||
case 'N':
|
||||
++buf.position();
|
||||
assertString("AN", buf);
|
||||
x = std::numeric_limits<T>::quiet_NaN();
|
||||
return;
|
||||
return ReturnType(parse_special_value("AN", std::numeric_limits<T>::quiet_NaN()));
|
||||
|
||||
default:
|
||||
if (negative)
|
||||
x = -x;
|
||||
return;
|
||||
return ReturnType(true);
|
||||
}
|
||||
++buf.position();
|
||||
}
|
||||
if (negative)
|
||||
x = -x;
|
||||
|
||||
return ReturnType(true);
|
||||
}
|
||||
|
||||
template <class T>
|
||||
inline bool tryReadFloatText(T & x, ReadBuffer & buf)
|
||||
{
|
||||
return readFloatTextImpl<T, bool>(x, buf);
|
||||
}
|
||||
|
||||
template <class T>
|
||||
inline void readFloatText(T & x, ReadBuffer & buf)
|
||||
{
|
||||
readFloatTextImpl<T, void>(x, buf);
|
||||
}
|
||||
|
||||
/// грубо; всё до '\n' или '\t'
|
||||
void readString(String & s, ReadBuffer & buf);
|
||||
@ -315,6 +378,8 @@ void readDoubleQuotedString(String & s, ReadBuffer & buf);
|
||||
|
||||
void readBackQuotedString(String & s, ReadBuffer & buf);
|
||||
|
||||
void readStringUntilEOF(String & s, ReadBuffer & buf);
|
||||
|
||||
|
||||
/// в формате YYYY-MM-DD
|
||||
inline void readDateText(DayNum_t & date, ReadBuffer & buf)
|
||||
@ -362,7 +427,7 @@ void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf);
|
||||
inline void readDateTimeText(time_t & datetime, ReadBuffer & buf)
|
||||
{
|
||||
/** Считываем 10 символов, которые могут быть unix timestamp.
|
||||
* При этом, поддерживается только unix timestamp из 10 символов - от 9 сентября 2001.
|
||||
* При этом, поддерживается только unix timestamp из 5-10 символов.
|
||||
* Потом смотрим на пятый символ. Если это число - парсим unix timestamp.
|
||||
* Если это не число - парсим YYYY-MM-DD hh:mm:ss.
|
||||
*/
|
||||
|
@ -968,6 +968,25 @@ protected:
|
||||
Table & table_dst,
|
||||
Table & table_src) const;
|
||||
|
||||
/// Слить данные из хэш-таблицы src в dst, но только для ключей, которые уже есть в dst. В остальных случаях, слить данные в overflows.
|
||||
template <typename Method, typename Table>
|
||||
void mergeDataNoMoreKeysImpl(
|
||||
Table & table_dst,
|
||||
AggregatedDataWithoutKey & overflows,
|
||||
Table & table_src) const;
|
||||
|
||||
/// То же самое, но игнорирует остальные ключи.
|
||||
template <typename Method, typename Table>
|
||||
void mergeDataOnlyExistingKeysImpl(
|
||||
Table & table_dst,
|
||||
Table & table_src) const;
|
||||
|
||||
/// Слить все ключи, оставшиеся после предыдущего метода, в overflows.
|
||||
template <typename Method, typename Table>
|
||||
void mergeDataRemainingKeysToOverflowsImpl(
|
||||
AggregatedDataWithoutKey & overflows,
|
||||
Table & table_src) const;
|
||||
|
||||
void mergeWithoutKeyDataImpl(
|
||||
ManyAggregatedDataVariants & non_empty_data) const;
|
||||
|
||||
@ -1024,13 +1043,24 @@ protected:
|
||||
bool final,
|
||||
boost::threadpool::pool * thread_pool) const;
|
||||
|
||||
template <bool no_more_keys, typename Method, typename Table>
|
||||
void mergeStreamsImplCase(
|
||||
Block & block,
|
||||
const Sizes & key_sizes,
|
||||
Arena * aggregates_pool,
|
||||
Method & method,
|
||||
Table & data,
|
||||
AggregateDataPtr overflow_row) const;
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void mergeStreamsImpl(
|
||||
Block & block,
|
||||
const Sizes & key_sizes,
|
||||
Arena * aggregates_pool,
|
||||
Method & method,
|
||||
Table & data) const;
|
||||
Table & data,
|
||||
AggregateDataPtr overflow_row,
|
||||
bool no_more_keys) const;
|
||||
|
||||
void mergeWithoutKeyStreamsImpl(
|
||||
Block & block,
|
||||
@ -1049,6 +1079,15 @@ protected:
|
||||
template <typename Method>
|
||||
void destroyImpl(
|
||||
Method & method) const;
|
||||
|
||||
|
||||
/** Проверяет ограничения на максимальное количество ключей для агрегации.
|
||||
* Если оно превышено, то, в зависимости от group_by_overflow_mode, либо
|
||||
* - кидает исключение;
|
||||
* - возвращает false, что говорит о том, что выполнение нужно прервать;
|
||||
* - выставляет переменную no_more_keys в true.
|
||||
*/
|
||||
bool checkLimits(size_t result_size, bool & no_more_keys) const;
|
||||
};
|
||||
|
||||
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
/// С локальными узлами соединение не устанавливается, а выполяется запрос напрямую.
|
||||
/// Поэтому храним только количество локальных узлов
|
||||
/// В конфиге кластер включает в себя узлы <node> или <shard>
|
||||
class Cluster : private boost::noncopyable
|
||||
class Cluster
|
||||
{
|
||||
public:
|
||||
Cluster(const Settings & settings, const String & cluster_name);
|
||||
@ -22,28 +22,13 @@ public:
|
||||
Cluster(const Settings & settings, std::vector<std::vector<String>> names,
|
||||
const String & username, const String & password);
|
||||
|
||||
/// количество узлов clickhouse сервера, расположенных локально
|
||||
/// к локальным узлам обращаемся напрямую
|
||||
size_t getLocalNodesNum() const { return local_nodes_num; }
|
||||
Cluster(const Cluster &) = delete;
|
||||
Cluster & operator=(const Cluster &) = delete;
|
||||
|
||||
/// используеться для выставления ограничения на размер таймаута
|
||||
static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit);
|
||||
|
||||
public:
|
||||
/// Соединения с удалёнными серверами.
|
||||
ConnectionPools pools;
|
||||
|
||||
struct ShardInfo
|
||||
{
|
||||
/// contains names of directories for asynchronous write to StorageDistributed
|
||||
std::vector<std::string> dir_names;
|
||||
UInt32 shard_num;
|
||||
int weight;
|
||||
size_t num_local_nodes;
|
||||
};
|
||||
std::vector<ShardInfo> shard_info_vec;
|
||||
std::vector<size_t> slot_to_shard;
|
||||
|
||||
struct Address
|
||||
{
|
||||
/** В конфиге адреса либо находятся в узлах <node>:
|
||||
@ -73,25 +58,59 @@ public:
|
||||
Address(const String & host_port_, const String & user_, const String & password_);
|
||||
};
|
||||
|
||||
private:
|
||||
static bool isLocal(const Address & address);
|
||||
using Addresses = std::vector<Address>;
|
||||
using AddressesWithFailover = std::vector<Addresses>;
|
||||
|
||||
struct ShardInfo
|
||||
{
|
||||
public:
|
||||
bool isLocal() const { return !local_addresses.empty(); }
|
||||
bool hasRemoteConnections() const { return !pool.isNull(); }
|
||||
size_t getLocalNodeCount() const { return local_addresses.size(); }
|
||||
|
||||
public:
|
||||
/// contains names of directories for asynchronous write to StorageDistributed
|
||||
std::vector<std::string> dir_names;
|
||||
UInt32 shard_num;
|
||||
int weight;
|
||||
Addresses local_addresses;
|
||||
mutable ConnectionPoolPtr pool;
|
||||
};
|
||||
|
||||
using ShardsInfo = std::vector<ShardInfo>;
|
||||
|
||||
public:
|
||||
const ShardsInfo & getShardsInfo() const { return shards_info; }
|
||||
const Addresses & getShardsAddresses() const { return addresses; }
|
||||
const AddressesWithFailover & getShardsWithFailoverAddresses() const { return addresses_with_failover; }
|
||||
|
||||
const ShardInfo * getAnyRemoteShardInfo() const { return any_remote_shard_info; }
|
||||
|
||||
/// Количество удалённых шардов.
|
||||
size_t getRemoteShardCount() const { return remote_shard_count; }
|
||||
|
||||
/// Количество узлов clickhouse сервера, расположенных локально
|
||||
/// к локальным узлам обращаемся напрямую.
|
||||
size_t getLocalShardCount() const { return local_shard_count; }
|
||||
|
||||
public:
|
||||
std::vector<size_t> slot_to_shard;
|
||||
|
||||
private:
|
||||
void initMisc();
|
||||
|
||||
private:
|
||||
/// Описание шардов кластера.
|
||||
ShardsInfo shards_info;
|
||||
/// Любой удалённый шард.
|
||||
ShardInfo * any_remote_shard_info = nullptr;
|
||||
/// Массив шардов. Каждый шард - адреса одного сервера.
|
||||
typedef std::vector<Address> Addresses;
|
||||
|
||||
/// Массив шардов. Для каждого шарда - массив адресов реплик (серверов, считающихся идентичными).
|
||||
typedef std::vector<Addresses> AddressesWithFailover;
|
||||
|
||||
public:
|
||||
const Addresses & getShardsInfo() const { return addresses; }
|
||||
const AddressesWithFailover & getShardsWithFailoverInfo() const { return addresses_with_failover; }
|
||||
|
||||
private:
|
||||
Addresses addresses;
|
||||
/// Массив шардов. Для каждого шарда - массив адресов реплик (серверов, считающихся идентичными).
|
||||
AddressesWithFailover addresses_with_failover;
|
||||
|
||||
size_t local_nodes_num = 0;
|
||||
size_t remote_shard_count = 0;
|
||||
size_t local_shard_count = 0;
|
||||
};
|
||||
|
||||
struct Clusters
|
||||
|
@ -182,6 +182,9 @@ private:
|
||||
/// Например, для ARRAY JOIN [1,2] AS b сюда попадет "b" -> "array(1,2)".
|
||||
NameToNameMap array_join_alias_to_name;
|
||||
|
||||
/// Обратное отображение для array_join_alias_to_name.
|
||||
NameToNameMap array_join_name_to_alias;
|
||||
|
||||
/// Нужно ли подготавливать к выполнению глобальные подзапросы при анализировании запроса.
|
||||
bool do_global;
|
||||
|
||||
|
@ -13,6 +13,9 @@ public:
|
||||
InterpreterCheckQuery(ASTPtr query_ptr_, Context & context_);
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
Block getSampleBlock() const;
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
|
@ -166,6 +166,8 @@ struct Settings
|
||||
/** Для запросов SELECT из реплицируемой таблицы, кидать исключение, если на реплике нет куска, записанного с кворумом; \
|
||||
* не читать куски, которые ещё не были записаны с кворумом. */ \
|
||||
M(SettingUInt64, select_sequential_consistency, 0) \
|
||||
/** Максимальное количество различных шардов и максимальное количество реплик одного шарда в функции remote. */ \
|
||||
M(SettingUInt64, table_function_remote_max_addresses, 1000) \
|
||||
|
||||
/// Всевозможные ограничения на выполнение запроса.
|
||||
Limits limits;
|
||||
|
@ -36,7 +36,7 @@ inline void evaluateMissingDefaults(Block & block,
|
||||
return;
|
||||
|
||||
/** ExpressionAnalyzer eliminates "unused" columns, in order to ensure their safety
|
||||
* we are going to operate on a copy instead of the original block */
|
||||
* we are going to operate on a copy instead of the original block */
|
||||
Block copy_block{block};
|
||||
/// evaluate default values for defaulted columns
|
||||
ExpressionAnalyzer{default_expr_list, context, {}, required_columns}.getActions(true)->execute(copy_block);
|
||||
|
@ -1,13 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IAST.h>
|
||||
#include <DB/Parsers/ASTQueryWithOutput.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ASTCheckQuery : public IAST
|
||||
struct ASTCheckQuery : public ASTQueryWithOutput
|
||||
{
|
||||
ASTCheckQuery(StringRange range_ = StringRange()) : IAST(range_) {};
|
||||
ASTCheckQuery(StringRange range_ = StringRange()) : ASTQueryWithOutput(range_) {};
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const override { return ("CheckQuery_" + database + "_" + table); };
|
||||
|
@ -1,13 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IParserBase.h>
|
||||
#include <DB/Parsers/ParserQueryWithOutput.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Запрос вида
|
||||
* CHECK [TABLE] [database.]table
|
||||
*/
|
||||
class ParserCheckQuery : public IParserBase
|
||||
class ParserCheckQuery : public ParserQueryWithOutput
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "ALTER query"; }
|
||||
|
21
dbms/include/DB/Parsers/ParserQueryWithOutput.h
Normal file
21
dbms/include/DB/Parsers/ParserQueryWithOutput.h
Normal file
@ -0,0 +1,21 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IParserBase.h>
|
||||
#include <DB/Parsers/CommonParsers.h>
|
||||
#include <DB/Parsers/ASTQueryWithOutput.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Парсер для запросов поддерживающих секцию FORMAT.
|
||||
*/
|
||||
class ParserQueryWithOutput : public IParserBase
|
||||
{
|
||||
protected:
|
||||
bool parseFormat(ASTQueryWithOutput & query, Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
|
||||
|
||||
protected:
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
};
|
||||
|
||||
}
|
@ -1,13 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IParserBase.h>
|
||||
#include <DB/Parsers/ParserQueryWithOutput.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class ParserSelectQuery : public IParserBase
|
||||
class ParserSelectQuery : public ParserQueryWithOutput
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "SELECT query"; }
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IParserBase.h>
|
||||
#include <DB/Parsers/ParserQueryWithOutput.h>
|
||||
#include <DB/Parsers/CommonParsers.h>
|
||||
#include <DB/Parsers/ExpressionElementParsers.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
@ -12,7 +13,7 @@ namespace DB
|
||||
|
||||
/** Запрос SHOW PROCESSLIST
|
||||
*/
|
||||
class ParserShowProcesslistQuery : public IParserBase
|
||||
class ParserShowProcesslistQuery : public ParserQueryWithOutput
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "SHOW PROCESSLIST query"; }
|
||||
@ -21,12 +22,11 @@ protected:
|
||||
{
|
||||
Pos begin = pos;
|
||||
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
ParserString s_show("SHOW", true, true);
|
||||
ParserString s_processlist("PROCESSLIST", true, true);
|
||||
ParserString s_format("FORMAT", true, true);
|
||||
|
||||
ASTPtr format;
|
||||
ASTShowProcesslistQuery * query = new ASTShowProcesslistQuery;
|
||||
ASTPtr query_ptr = query;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
@ -40,22 +40,12 @@ protected:
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (s_format.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
/// FORMAT format_name
|
||||
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
ParserIdentifier format_p;
|
||||
|
||||
if (!format_p.parse(pos, end, format, max_parsed_pos, expected))
|
||||
return false;
|
||||
typeid_cast<ASTIdentifier &>(*format).kind = ASTIdentifier::Format;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
ASTShowProcesslistQuery * query = new ASTShowProcesslistQuery(StringRange(begin, pos));
|
||||
query->format = format;
|
||||
node = query;
|
||||
query->range = StringRange(begin, pos);
|
||||
node = query_ptr;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IParserBase.h>
|
||||
#include <DB/Parsers/ParserQueryWithOutput.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -11,7 +11,7 @@ namespace DB
|
||||
* или
|
||||
* SHOW DATABASES.
|
||||
*/
|
||||
class ParserShowTablesQuery : public IParserBase
|
||||
class ParserShowTablesQuery : public ParserQueryWithOutput
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "SHOW TABLES|DATABASES query"; }
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IParserBase.h>
|
||||
#include <DB/Parsers/ParserQueryWithOutput.h>
|
||||
#include <DB/Parsers/ExpressionElementParsers.h>
|
||||
|
||||
|
||||
@ -9,7 +10,7 @@ namespace DB
|
||||
|
||||
/** Запрос (EXISTS | SHOW CREATE | (DESCRIBE | DESC) ) [TABLE] [db.]name [FORMAT format]
|
||||
*/
|
||||
class ParserTablePropertiesQuery : public IParserBase
|
||||
class ParserTablePropertiesQuery : public ParserQueryWithOutput
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "EXISTS, SHOW CREATE or DESCRIBE query"; }
|
||||
|
@ -39,7 +39,7 @@ public:
|
||||
|
||||
void write(const Block & block) override
|
||||
{
|
||||
if (storage.getShardingKeyExpr() && storage.cluster.shard_info_vec.size() > 1)
|
||||
if (storage.getShardingKeyExpr() && (storage.cluster.getShardsInfo().size() > 1))
|
||||
return writeSplit(block);
|
||||
|
||||
writeImpl(block);
|
||||
@ -50,7 +50,7 @@ private:
|
||||
static std::vector<IColumn::Filter> createFiltersImpl(const size_t num_rows, const IColumn * column, const Cluster & cluster)
|
||||
{
|
||||
const auto total_weight = cluster.slot_to_shard.size();
|
||||
const auto num_shards = cluster.shard_info_vec.size();
|
||||
const auto num_shards = cluster.getShardsInfo().size();
|
||||
std::vector<IColumn::Filter> filters(num_shards);
|
||||
|
||||
/** Деление отрицательного числа с остатком на положительное, в C++ даёт отрицательный остаток.
|
||||
@ -123,7 +123,7 @@ private:
|
||||
|
||||
auto filters = createFilters(block);
|
||||
|
||||
const auto num_shards = storage.cluster.shard_info_vec.size();
|
||||
const auto num_shards = storage.cluster.getShardsInfo().size();
|
||||
for (size_t i = 0; i < num_shards; ++i)
|
||||
{
|
||||
auto target_block = block.cloneEmpty();
|
||||
@ -138,9 +138,9 @@ private:
|
||||
|
||||
void writeImpl(const Block & block, const size_t shard_id = 0)
|
||||
{
|
||||
const auto & shard_info = storage.cluster.shard_info_vec[shard_id];
|
||||
if (shard_info.num_local_nodes)
|
||||
writeToLocal(block, shard_info.num_local_nodes);
|
||||
const auto & shard_info = storage.cluster.getShardsInfo()[shard_id];
|
||||
if (shard_info.getLocalNodeCount() > 0)
|
||||
writeToLocal(block, shard_info.getLocalNodeCount());
|
||||
|
||||
/// dir_names is empty if shard has only local addresses
|
||||
if (!shard_info.dir_names.empty())
|
||||
|
@ -99,10 +99,9 @@ public:
|
||||
private:
|
||||
virtual const NamesAndTypesList & getColumnsListImpl() const = 0;
|
||||
|
||||
using ColumnsListRange = boost::range::joined_range<
|
||||
const boost::iterator_range<NamesAndTypesList::const_iterator>,
|
||||
const boost::iterator_range<NamesAndTypesList::const_iterator>>;
|
||||
ColumnsListRange getColumnsListIterator() const;
|
||||
using ColumnsListRange = boost::range::joined_range<const NamesAndTypesList, const NamesAndTypesList>;
|
||||
/// Returns a lazily joined range of table's ordinary and materialized columns, without unnecessary copying
|
||||
ColumnsListRange getColumnsListRange() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -76,6 +76,9 @@ public:
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
/// От каждой реплики получить описание соответствующей локальной таблицы.
|
||||
BlockInputStreams describe(const Context & context, const Settings & settings);
|
||||
|
||||
const ExpressionActionsPtr & getShardingKeyExpr() const { return sharding_key_expr; }
|
||||
const String & getShardingKeyColumnName() const { return sharding_key_column_name; }
|
||||
size_t getShardCount() const;
|
||||
|
@ -24,9 +24,6 @@ namespace DB
|
||||
class TableFunctionRemote : public ITableFunction
|
||||
{
|
||||
public:
|
||||
/// Максимальное количество различных шардов и максимальное количество реплик одного шарда
|
||||
const size_t MAX_ADDRESSES = 1000; /// TODO Перенести в Settings.
|
||||
|
||||
std::string getName() const override { return "remote"; }
|
||||
|
||||
StoragePtr execute(ASTPtr ast_function, Context & context) const override
|
||||
@ -109,11 +106,13 @@ public:
|
||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get()))
|
||||
id->kind = ASTIdentifier::Table;
|
||||
|
||||
size_t max_addresses = context.getSettingsRef().table_function_remote_max_addresses;
|
||||
|
||||
std::vector<std::vector<String>> names;
|
||||
std::vector<String> shards = parseDescription(description, 0, description.size(), ',');
|
||||
std::vector<String> shards = parseDescription(description, 0, description.size(), ',', max_addresses);
|
||||
|
||||
for (size_t i = 0; i < shards.size(); ++i)
|
||||
names.push_back(parseDescription(shards[i], 0, shards[i].size(), '|'));
|
||||
names.push_back(parseDescription(shards[i], 0, shards[i].size(), '|', max_addresses));
|
||||
|
||||
if (names.empty())
|
||||
throw Exception("Shard list is empty after parsing first argument", ErrorCodes::BAD_ARGUMENTS);
|
||||
@ -133,10 +132,15 @@ private:
|
||||
Settings settings = context.getSettings();
|
||||
NamesAndTypesList res;
|
||||
|
||||
/// Отправляем на первый попавшийся шард
|
||||
/// Отправляем на первый попавшийся удалённый шард.
|
||||
const auto shard_info = cluster.getAnyRemoteShardInfo();
|
||||
if (shard_info == nullptr)
|
||||
throw Exception("No remote shard found", ErrorCodes::NO_REMOTE_SHARD_FOUND);
|
||||
ConnectionPoolPtr pool = shard_info->pool;
|
||||
|
||||
BlockInputStreamPtr input{
|
||||
new RemoteBlockInputStream{
|
||||
cluster.pools.front().get(), query, &settings, nullptr,
|
||||
pool.get(), query, &settings, nullptr,
|
||||
Tables(), QueryProcessingStage::Complete, context}
|
||||
};
|
||||
input->readPrefix();
|
||||
@ -164,7 +168,7 @@ private:
|
||||
}
|
||||
|
||||
/// Декартово произведение двух множеств строк, результат записываем на место первого аргумента
|
||||
void append(std::vector<String> & to, const std::vector<String> & what) const
|
||||
void append(std::vector<String> & to, const std::vector<String> & what, size_t max_addresses) const
|
||||
{
|
||||
if (what.empty()) return;
|
||||
if (to.empty())
|
||||
@ -172,7 +176,7 @@ private:
|
||||
to = what;
|
||||
return;
|
||||
}
|
||||
if (what.size() * to.size() > MAX_ADDRESSES)
|
||||
if (what.size() * to.size() > max_addresses)
|
||||
throw Exception("Storage Distributed, first argument generates too many result addresses",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
std::vector<String> res;
|
||||
@ -209,7 +213,7 @@ private:
|
||||
* abc{1..9}de{f,g,h} - прямое произведение, 27 шардов.
|
||||
* abc{1..9}de{0|1} - прямое произведение, 9 шардов, в каждом 2 реплики.
|
||||
*/
|
||||
std::vector<String> parseDescription(const String & description, size_t l, size_t r, char separator) const
|
||||
std::vector<String> parseDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses) const
|
||||
{
|
||||
std::vector<String> res;
|
||||
std::vector<String> cur;
|
||||
@ -263,7 +267,7 @@ private:
|
||||
throw Exception("Storage Distributed, incorrect argument in braces (left number is greater then right): "
|
||||
+ description.substr(i, m - i + 1),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (right - left + 1 > MAX_ADDRESSES)
|
||||
if (right - left + 1 > max_addresses)
|
||||
throw Exception("Storage Distributed, first argument generates too many result addresses",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
bool add_leading_zeroes = false;
|
||||
@ -282,25 +286,29 @@ private:
|
||||
buffer.push_back(cur);
|
||||
}
|
||||
} else if (have_splitter) /// Если внутри есть текущий разделитель, то сгенерировать множество получаемых строк
|
||||
buffer = parseDescription(description, i + 1, m, separator);
|
||||
buffer = parseDescription(description, i + 1, m, separator, max_addresses);
|
||||
else /// Иначе просто скопировать, порождение произойдет при вызове с правильным разделителем
|
||||
buffer.push_back(description.substr(i, m - i + 1));
|
||||
/// К текущему множеству строк добавить все возможные полученные продолжения
|
||||
append(cur, buffer);
|
||||
append(cur, buffer, max_addresses);
|
||||
i = m;
|
||||
} else if (description[i] == separator) {
|
||||
}
|
||||
else if (description[i] == separator)
|
||||
{
|
||||
/// Если разделитель, то добавляем в ответ найденные строки
|
||||
res.insert(res.end(), cur.begin(), cur.end());
|
||||
cur.clear();
|
||||
} else {
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Иначе просто дописываем символ к текущим строкам
|
||||
std::vector<String> buffer;
|
||||
buffer.push_back(description.substr(i, 1));
|
||||
append(cur, buffer);
|
||||
append(cur, buffer, max_addresses);
|
||||
}
|
||||
}
|
||||
res.insert(res.end(), cur.begin(), cur.end());
|
||||
if (res.size() > MAX_ADDRESSES)
|
||||
if (res.size() > max_addresses)
|
||||
throw Exception("Storage Distributed, first argument generates too many result addresses",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
return res;
|
||||
|
@ -142,7 +142,7 @@ private:
|
||||
void run()
|
||||
{
|
||||
for (size_t i = 0; i < concurrency; ++i)
|
||||
pool.schedule(std::bind(&Benchmark::thread, this, connections.get()));
|
||||
pool.schedule(std::bind(&Benchmark::thread, this, connections.IConnectionPool::get()));
|
||||
|
||||
InterruptListener interrupt_listener;
|
||||
|
||||
|
@ -536,5 +536,13 @@ BlockStreamProfileInfo Connection::receiveProfileInfo()
|
||||
return profile_info;
|
||||
}
|
||||
|
||||
void Connection::fillBlockExtraInfo(BlockExtraInfo & info) const
|
||||
{
|
||||
info.is_valid = true;
|
||||
info.host = host;
|
||||
info.resolved_address = resolved_address.toString();
|
||||
info.port = port;
|
||||
info.user = user;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -11,16 +11,17 @@ ParallelReplicas::ParallelReplicas(Connection * connection_, const Settings * se
|
||||
registerReplica(connection_);
|
||||
}
|
||||
|
||||
ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_)
|
||||
ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_,
|
||||
bool append_extra_info, bool get_all_replicas)
|
||||
: settings(settings_), throttler(throttler_)
|
||||
{
|
||||
if (pool_ == nullptr)
|
||||
throw Exception("Null pool specified", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
bool has_many_replicas = (settings != nullptr) && (settings->max_parallel_replicas > 1);
|
||||
bool has_many_replicas = get_all_replicas || ((settings != nullptr) && (settings->max_parallel_replicas > 1));
|
||||
if (has_many_replicas)
|
||||
{
|
||||
pool_entries = pool_->getMany(settings);
|
||||
pool_entries = pool_->getMany(settings, get_all_replicas);
|
||||
active_replica_count = pool_entries.size();
|
||||
supports_parallel_execution = (active_replica_count > 1);
|
||||
|
||||
@ -40,6 +41,9 @@ ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, const Settings * set
|
||||
if (!pool_entry.isNull())
|
||||
registerReplica(&*pool_entry);
|
||||
}
|
||||
|
||||
if (append_extra_info)
|
||||
block_extra_info.reset(new BlockExtraInfo);
|
||||
}
|
||||
|
||||
void ParallelReplicas::sendExternalTablesData(std::vector<ExternalTablesData> & data)
|
||||
@ -104,7 +108,23 @@ void ParallelReplicas::sendQuery(const String & query, const String & query_id,
|
||||
Connection::Packet ParallelReplicas::receivePacket()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
|
||||
return receivePacketUnlocked();
|
||||
const auto & packet = receivePacketUnlocked();
|
||||
if (block_extra_info)
|
||||
{
|
||||
if (packet.type == Protocol::Server::Data)
|
||||
current_connection->fillBlockExtraInfo(*block_extra_info);
|
||||
else
|
||||
block_extra_info->is_valid = false;
|
||||
}
|
||||
return packet;
|
||||
}
|
||||
|
||||
BlockExtraInfo ParallelReplicas::getBlockExtraInfo() const
|
||||
{
|
||||
if (!block_extra_info)
|
||||
throw Exception("ParallelReplicas object not configured for block extra info support",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
return *block_extra_info;
|
||||
}
|
||||
|
||||
void ParallelReplicas::disconnect()
|
||||
@ -219,8 +239,8 @@ Connection::Packet ParallelReplicas::receivePacketUnlocked()
|
||||
if (it == replica_map.end())
|
||||
throw Exception("Logical error: no available replica", ErrorCodes::NO_AVAILABLE_REPLICA);
|
||||
|
||||
Connection * connection = it->second;
|
||||
Connection::Packet packet = connection->receivePacket();
|
||||
current_connection = it->second;
|
||||
Connection::Packet packet = current_connection->receivePacket();
|
||||
|
||||
switch (packet.type)
|
||||
{
|
||||
@ -237,7 +257,7 @@ Connection::Packet ParallelReplicas::receivePacketUnlocked()
|
||||
|
||||
case Protocol::Server::Exception:
|
||||
default:
|
||||
connection->disconnect();
|
||||
current_connection->disconnect();
|
||||
invalidateReplica(it);
|
||||
break;
|
||||
}
|
||||
|
261
dbms/src/Common/tests/compact_array.cpp
Normal file
261
dbms/src/Common/tests/compact_array.cpp
Normal file
@ -0,0 +1,261 @@
|
||||
#include <DB/Common/CompactArray.h>
|
||||
#include <DB/IO/WriteBufferFromFile.h>
|
||||
#include <DB/IO/ReadBufferFromFile.h>
|
||||
|
||||
#include <boost/filesystem.hpp>
|
||||
|
||||
#include <string>
|
||||
#include <iostream>
|
||||
#include <fstream>
|
||||
#include <stdexcept>
|
||||
#include <unistd.h>
|
||||
#include <cstdlib>
|
||||
|
||||
namespace fs = boost::filesystem;
|
||||
|
||||
std::string createTmpPath(const std::string & filename)
|
||||
{
|
||||
char pattern[] = "/tmp/fileXXXXXX";
|
||||
char * dir = mkdtemp(pattern);
|
||||
if (dir == nullptr)
|
||||
throw std::runtime_error("Could not create directory");
|
||||
|
||||
return std::string(dir) + "/" + filename;
|
||||
}
|
||||
|
||||
template <size_t width, size_t bucket_count, typename Generator>
|
||||
struct Test
|
||||
{
|
||||
static void perform()
|
||||
{
|
||||
bool ok = true;
|
||||
|
||||
std::string filename;
|
||||
|
||||
try
|
||||
{
|
||||
using Store = DB::CompactArray<UInt64, width, bucket_count>;
|
||||
|
||||
Store store;
|
||||
|
||||
for (size_t i = 0; i < bucket_count; ++i)
|
||||
store[i] = Generator::execute(i, width);
|
||||
|
||||
filename = createTmpPath("compact_array.bin");
|
||||
|
||||
{
|
||||
DB::WriteBufferFromFile wb(filename);
|
||||
wb.write(reinterpret_cast<const char *>(&store), sizeof(store));
|
||||
const unsigned char * p = reinterpret_cast<const unsigned char *>(&store);
|
||||
for (size_t i = 0; i < sizeof(store); ++i)
|
||||
++p;
|
||||
}
|
||||
|
||||
{
|
||||
DB::ReadBufferFromFile rb(filename);
|
||||
typename Store::Reader reader(rb);
|
||||
while (reader.next())
|
||||
{
|
||||
const auto & data = reader.get();
|
||||
if (data.second != store[data.first])
|
||||
throw std::runtime_error("Found discrepancy");
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (const Poco::Exception & ex)
|
||||
{
|
||||
std::cout << "Test width=" << width << " bucket_count=" << bucket_count << " failed "
|
||||
<< "(Error: " << ex.what() << ": " << ex.displayText() << ")\n";
|
||||
ok = false;
|
||||
}
|
||||
catch (const std::runtime_error & ex)
|
||||
{
|
||||
std::cout << "Test width=" << width << " bucket_count=" << bucket_count << " failed "
|
||||
<< "(Error: " << ex.what() << ")\n";
|
||||
ok = false;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
std::cout << "Test width=" << width << " bucket_count=" << bucket_count << " failed\n";
|
||||
ok = false;
|
||||
}
|
||||
|
||||
fs::remove_all(fs::path(filename).parent_path().string());
|
||||
|
||||
if (ok)
|
||||
std::cout << "Test width=" << width << " bucket_count=" << bucket_count << " passed\n";
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Generator>
|
||||
struct TestSet
|
||||
{
|
||||
static void execute()
|
||||
{
|
||||
Test<1, 1, Generator>::perform();
|
||||
Test<1, 2, Generator>::perform();
|
||||
Test<1, 3, Generator>::perform();
|
||||
Test<1, 4, Generator>::perform();
|
||||
Test<1, 5, Generator>::perform();
|
||||
Test<1, 6, Generator>::perform();
|
||||
Test<1, 7, Generator>::perform();
|
||||
Test<1, 8, Generator>::perform();
|
||||
Test<1, 9, Generator>::perform();
|
||||
Test<1, 10, Generator>::perform();
|
||||
Test<1, 16, Generator>::perform();
|
||||
Test<1, 32, Generator>::perform();
|
||||
Test<1, 64, Generator>::perform();
|
||||
Test<1, 128, Generator>::perform();
|
||||
Test<1, 256, Generator>::perform();
|
||||
Test<1, 512, Generator>::perform();
|
||||
Test<1, 1024, Generator>::perform();
|
||||
|
||||
Test<2, 1, Generator>::perform();
|
||||
Test<2, 2, Generator>::perform();
|
||||
Test<2, 3, Generator>::perform();
|
||||
Test<2, 4, Generator>::perform();
|
||||
Test<2, 5, Generator>::perform();
|
||||
Test<2, 6, Generator>::perform();
|
||||
Test<2, 7, Generator>::perform();
|
||||
Test<2, 8, Generator>::perform();
|
||||
Test<2, 9, Generator>::perform();
|
||||
Test<2, 10, Generator>::perform();
|
||||
Test<2, 16, Generator>::perform();
|
||||
Test<2, 32, Generator>::perform();
|
||||
Test<2, 64, Generator>::perform();
|
||||
Test<2, 128, Generator>::perform();
|
||||
Test<2, 256, Generator>::perform();
|
||||
Test<2, 512, Generator>::perform();
|
||||
Test<2, 1024, Generator>::perform();
|
||||
|
||||
Test<3, 1, Generator>::perform();
|
||||
Test<3, 2, Generator>::perform();
|
||||
Test<3, 3, Generator>::perform();
|
||||
Test<3, 4, Generator>::perform();
|
||||
Test<3, 5, Generator>::perform();
|
||||
Test<3, 6, Generator>::perform();
|
||||
Test<3, 7, Generator>::perform();
|
||||
Test<3, 8, Generator>::perform();
|
||||
Test<3, 9, Generator>::perform();
|
||||
Test<3, 10, Generator>::perform();
|
||||
Test<3, 16, Generator>::perform();
|
||||
Test<3, 32, Generator>::perform();
|
||||
Test<3, 64, Generator>::perform();
|
||||
Test<3, 128, Generator>::perform();
|
||||
Test<3, 256, Generator>::perform();
|
||||
Test<3, 512, Generator>::perform();
|
||||
Test<3, 1024, Generator>::perform();
|
||||
|
||||
Test<4, 1, Generator>::perform();
|
||||
Test<4, 2, Generator>::perform();
|
||||
Test<4, 3, Generator>::perform();
|
||||
Test<4, 4, Generator>::perform();
|
||||
Test<4, 5, Generator>::perform();
|
||||
Test<4, 6, Generator>::perform();
|
||||
Test<4, 7, Generator>::perform();
|
||||
Test<4, 8, Generator>::perform();
|
||||
Test<4, 9, Generator>::perform();
|
||||
Test<4, 10, Generator>::perform();
|
||||
Test<4, 16, Generator>::perform();
|
||||
Test<4, 32, Generator>::perform();
|
||||
Test<4, 64, Generator>::perform();
|
||||
Test<4, 128, Generator>::perform();
|
||||
Test<4, 256, Generator>::perform();
|
||||
Test<4, 512, Generator>::perform();
|
||||
Test<4, 1024, Generator>::perform();
|
||||
|
||||
Test<5, 1, Generator>::perform();
|
||||
Test<5, 2, Generator>::perform();
|
||||
Test<5, 3, Generator>::perform();
|
||||
Test<5, 4, Generator>::perform();
|
||||
Test<5, 5, Generator>::perform();
|
||||
Test<5, 6, Generator>::perform();
|
||||
Test<5, 7, Generator>::perform();
|
||||
Test<5, 8, Generator>::perform();
|
||||
Test<5, 9, Generator>::perform();
|
||||
Test<5, 10, Generator>::perform();
|
||||
Test<5, 16, Generator>::perform();
|
||||
Test<5, 32, Generator>::perform();
|
||||
Test<5, 64, Generator>::perform();
|
||||
Test<5, 128, Generator>::perform();
|
||||
Test<5, 256, Generator>::perform();
|
||||
Test<5, 512, Generator>::perform();
|
||||
Test<5, 1024, Generator>::perform();
|
||||
|
||||
Test<6, 1, Generator>::perform();
|
||||
Test<6, 2, Generator>::perform();
|
||||
Test<6, 3, Generator>::perform();
|
||||
Test<6, 4, Generator>::perform();
|
||||
Test<6, 5, Generator>::perform();
|
||||
Test<6, 6, Generator>::perform();
|
||||
Test<6, 7, Generator>::perform();
|
||||
Test<6, 8, Generator>::perform();
|
||||
Test<6, 9, Generator>::perform();
|
||||
Test<6, 10, Generator>::perform();
|
||||
Test<6, 16, Generator>::perform();
|
||||
Test<6, 32, Generator>::perform();
|
||||
Test<6, 64, Generator>::perform();
|
||||
Test<6, 128, Generator>::perform();
|
||||
Test<6, 256, Generator>::perform();
|
||||
Test<6, 512, Generator>::perform();
|
||||
Test<6, 1024, Generator>::perform();
|
||||
|
||||
Test<7, 1, Generator>::perform();
|
||||
Test<7, 2, Generator>::perform();
|
||||
Test<7, 3, Generator>::perform();
|
||||
Test<7, 4, Generator>::perform();
|
||||
Test<7, 5, Generator>::perform();
|
||||
Test<7, 6, Generator>::perform();
|
||||
Test<7, 7, Generator>::perform();
|
||||
Test<7, 8, Generator>::perform();
|
||||
Test<7, 9, Generator>::perform();
|
||||
Test<7, 10, Generator>::perform();
|
||||
Test<7, 16, Generator>::perform();
|
||||
Test<7, 32, Generator>::perform();
|
||||
Test<7, 64, Generator>::perform();
|
||||
Test<7, 128, Generator>::perform();
|
||||
Test<7, 256, Generator>::perform();
|
||||
Test<7, 512, Generator>::perform();
|
||||
Test<7, 1024, Generator>::perform();
|
||||
}
|
||||
};
|
||||
|
||||
struct Generator1
|
||||
{
|
||||
static UInt8 execute(size_t i, size_t width)
|
||||
{
|
||||
return (1 << width) - 1;
|
||||
}
|
||||
};
|
||||
|
||||
struct Generator2
|
||||
{
|
||||
static UInt8 execute(size_t i, size_t width)
|
||||
{
|
||||
return (i >> 1) & ((1 << width) - 1);
|
||||
}
|
||||
};
|
||||
|
||||
struct Generator3
|
||||
{
|
||||
static UInt8 execute(size_t i, size_t width)
|
||||
{
|
||||
return (i * 17 + 31) % (1ULL << width);
|
||||
}
|
||||
};
|
||||
|
||||
void runTests()
|
||||
{
|
||||
std::cout << "Test set 1\n";
|
||||
TestSet<Generator1>::execute();
|
||||
std::cout << "Test set 2\n";
|
||||
TestSet<Generator2>::execute();
|
||||
std::cout << "Test set 3\n";
|
||||
TestSet<Generator3>::execute();
|
||||
}
|
||||
|
||||
int main()
|
||||
{
|
||||
runTests();
|
||||
return 0;
|
||||
}
|
@ -10,47 +10,170 @@
|
||||
|
||||
#include <DB/DataStreams/LimitBlockInputStream.h>
|
||||
#include <DB/DataStreams/UnionBlockInputStream.h>
|
||||
#include <DB/DataStreams/BlockExtraInfoInputStream.h>
|
||||
#include <DB/DataStreams/FormatFactory.h>
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
void test1()
|
||||
{
|
||||
DB::StoragePtr table = DB::StorageSystemNumbers::create("numbers");
|
||||
|
||||
DB::Names column_names;
|
||||
column_names.push_back("number");
|
||||
|
||||
DB::QueryProcessingStage::Enum stage1;
|
||||
DB::QueryProcessingStage::Enum stage2;
|
||||
DB::QueryProcessingStage::Enum stage3;
|
||||
|
||||
DB::BlockInputStreams streams;
|
||||
streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage1, 1)[0], 30, 30000));
|
||||
streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage2, 1)[0], 30, 2000));
|
||||
streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage3, 1)[0], 30, 100));
|
||||
|
||||
DB::UnionBlockInputStream<> union_stream(streams, nullptr, 2);
|
||||
|
||||
DB::FormatFactory format_factory;
|
||||
DB::WriteBufferFromFileDescriptor wb(STDERR_FILENO);
|
||||
DB::Block sample = table->getSampleBlock();
|
||||
DB::BlockOutputStreamPtr out = format_factory.getOutput("TabSeparated", wb, sample);
|
||||
|
||||
while (DB::Block block = union_stream.read())
|
||||
{
|
||||
out->write(block);
|
||||
wb.next();
|
||||
}
|
||||
//DB::copyData(union_stream, *out);
|
||||
}
|
||||
|
||||
void test2()
|
||||
{
|
||||
DB::StoragePtr table = DB::StorageSystemNumbers::create("numbers");
|
||||
|
||||
DB::Names column_names;
|
||||
column_names.push_back("number");
|
||||
|
||||
DB::QueryProcessingStage::Enum stage1;
|
||||
DB::QueryProcessingStage::Enum stage2;
|
||||
DB::QueryProcessingStage::Enum stage3;
|
||||
|
||||
DB::BlockExtraInfo extra_info1;
|
||||
extra_info1.host = "host1";
|
||||
extra_info1.resolved_address = "127.0.0.1";
|
||||
extra_info1.port = 9000;
|
||||
extra_info1.user = "user1";
|
||||
|
||||
DB::BlockExtraInfo extra_info2;
|
||||
extra_info2.host = "host2";
|
||||
extra_info2.resolved_address = "127.0.0.2";
|
||||
extra_info2.port = 9001;
|
||||
extra_info2.user = "user2";
|
||||
|
||||
DB::BlockExtraInfo extra_info3;
|
||||
extra_info3.host = "host3";
|
||||
extra_info3.resolved_address = "127.0.0.3";
|
||||
extra_info3.port = 9003;
|
||||
extra_info3.user = "user3";
|
||||
|
||||
DB::BlockInputStreams streams;
|
||||
|
||||
DB::BlockInputStreamPtr stream1 = new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage1, 1)[0], 30, 30000);
|
||||
stream1 = new DB::BlockExtraInfoInputStream(stream1, extra_info1);
|
||||
streams.emplace_back(stream1);
|
||||
|
||||
DB::BlockInputStreamPtr stream2 = new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage2, 1)[0], 30, 2000);
|
||||
stream2 = new DB::BlockExtraInfoInputStream(stream2, extra_info2);
|
||||
streams.emplace_back(stream2);
|
||||
|
||||
DB::BlockInputStreamPtr stream3 = new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage3, 1)[0], 30, 100);
|
||||
stream3 = new DB::BlockExtraInfoInputStream(stream3, extra_info3);
|
||||
streams.emplace_back(stream3);
|
||||
|
||||
DB::UnionBlockInputStream<DB::StreamUnionMode::ExtraInfo> union_stream(streams, nullptr, 2);
|
||||
|
||||
auto getSampleBlock = []()
|
||||
{
|
||||
DB::Block block;
|
||||
DB::ColumnWithTypeAndName col;
|
||||
|
||||
col.name = "number";
|
||||
col.type = new DB::DataTypeUInt64;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "host_name";
|
||||
col.type = new DB::DataTypeString;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "host_address";
|
||||
col.type = new DB::DataTypeString;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "port";
|
||||
col.type = new DB::DataTypeUInt16;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "user";
|
||||
col.type = new DB::DataTypeString;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
return block;
|
||||
};
|
||||
|
||||
DB::FormatFactory format_factory;
|
||||
DB::WriteBufferFromFileDescriptor wb(STDERR_FILENO);
|
||||
DB::Block sample = getSampleBlock();
|
||||
DB::BlockOutputStreamPtr out = format_factory.getOutput("TabSeparated", wb, sample);
|
||||
|
||||
while (DB::Block block = union_stream.read())
|
||||
{
|
||||
const auto & col = block.getByPosition(0);
|
||||
auto extra_info = union_stream.getBlockExtraInfo();
|
||||
|
||||
DB::ColumnPtr host_name_column = new DB::ColumnString;
|
||||
DB::ColumnPtr host_address_column = new DB::ColumnString;
|
||||
DB::ColumnPtr port_column = new DB::ColumnUInt16;
|
||||
DB::ColumnPtr user_column = new DB::ColumnString;
|
||||
|
||||
size_t row_count = block.rows();
|
||||
for (size_t i = 0; i < row_count; ++i)
|
||||
{
|
||||
host_name_column->insert(extra_info.resolved_address);
|
||||
host_address_column->insert(extra_info.host);
|
||||
port_column->insert(static_cast<UInt64>(extra_info.port));
|
||||
user_column->insert(extra_info.user);
|
||||
}
|
||||
|
||||
DB::Block out_block;
|
||||
out_block.insert(DB::ColumnWithTypeAndName(col.column->clone(), col.type, col.name));
|
||||
out_block.insert(DB::ColumnWithTypeAndName(host_name_column, new DB::DataTypeString, "host_name"));
|
||||
out_block.insert(DB::ColumnWithTypeAndName(host_address_column, new DB::DataTypeString, "host_address"));
|
||||
out_block.insert(DB::ColumnWithTypeAndName(port_column, new DB::DataTypeUInt16, "port"));
|
||||
out_block.insert(DB::ColumnWithTypeAndName(user_column, new DB::DataTypeString, "user"));
|
||||
|
||||
out->write(out_block);
|
||||
wb.next();
|
||||
}
|
||||
//DB::copyData(union_stream, *out);
|
||||
}
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
try
|
||||
{
|
||||
DB::StoragePtr table = DB::StorageSystemNumbers::create("numbers");
|
||||
|
||||
DB::Names column_names;
|
||||
column_names.push_back("number");
|
||||
|
||||
DB::QueryProcessingStage::Enum stage1;
|
||||
DB::QueryProcessingStage::Enum stage2;
|
||||
DB::QueryProcessingStage::Enum stage3;
|
||||
|
||||
DB::BlockInputStreams streams;
|
||||
streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage1, 1)[0], 30, 30000));
|
||||
streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage2, 1)[0], 30, 2000));
|
||||
streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage3, 1)[0], 30, 100));
|
||||
|
||||
DB::UnionBlockInputStream union_stream(streams, nullptr, 2);
|
||||
|
||||
DB::FormatFactory format_factory;
|
||||
DB::WriteBufferFromFileDescriptor wb(STDERR_FILENO);
|
||||
DB::Block sample = table->getSampleBlock();
|
||||
DB::BlockOutputStreamPtr out = format_factory.getOutput("TabSeparated", wb, sample);
|
||||
|
||||
while (DB::Block block = union_stream.read())
|
||||
{
|
||||
out->write(block);
|
||||
wb.next();
|
||||
}
|
||||
//DB::copyData(union_stream, *out);
|
||||
test1();
|
||||
test2();
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
|
@ -44,7 +44,7 @@ int main(int argc, char ** argv)
|
||||
for (size_t i = 0, size = streams.size(); i < size; ++i)
|
||||
streams[i] = new DB::AsynchronousBlockInputStream(streams[i]);
|
||||
|
||||
DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream(streams, nullptr, settings.max_threads);
|
||||
DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream<>(streams, nullptr, settings.max_threads);
|
||||
stream = new DB::LimitBlockInputStream(stream, 10, 0);
|
||||
|
||||
DB::FormatFactory format_factory;
|
||||
|
@ -17,6 +17,7 @@ void registerFunctionsString(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionReverse>();
|
||||
factory.registerFunction<FunctionReverseUTF8>();
|
||||
factory.registerFunction<FunctionConcat>();
|
||||
factory.registerFunction<FunctionConcatAssumeInjective>();
|
||||
factory.registerFunction<FunctionSubstring>();
|
||||
factory.registerFunction<FunctionSubstringUTF8>();
|
||||
factory.registerFunction<FunctionAppendTrailingCharIfAbsent>();
|
||||
|
@ -10,6 +10,7 @@ void registerFunctionsStringArray(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionAlphaTokens>();
|
||||
factory.registerFunction<FunctionSplitByChar>();
|
||||
factory.registerFunction<FunctionSplitByString>();
|
||||
factory.registerFunction<FunctionArrayStringConcat>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -26,14 +26,21 @@ static void __attribute__((__noinline__)) throwAtAssertionFailed(const char * s,
|
||||
}
|
||||
|
||||
|
||||
void assertString(const char * s, ReadBuffer & buf)
|
||||
bool checkString(const char * s, ReadBuffer & buf)
|
||||
{
|
||||
for (; *s; ++s)
|
||||
{
|
||||
if (buf.eof() || *buf.position() != *s)
|
||||
throwAtAssertionFailed(s, buf);
|
||||
return false;
|
||||
++buf.position();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
void assertString(const char * s, ReadBuffer & buf)
|
||||
{
|
||||
if (!checkString(s, buf))
|
||||
throwAtAssertionFailed(s, buf);
|
||||
}
|
||||
|
||||
void assertChar(char symbol, ReadBuffer & buf)
|
||||
@ -70,6 +77,20 @@ void readString(String & s, ReadBuffer & buf)
|
||||
}
|
||||
}
|
||||
|
||||
void readStringUntilEOF(String & s, ReadBuffer & buf)
|
||||
{
|
||||
s = "";
|
||||
while (!buf.eof())
|
||||
{
|
||||
size_t bytes = buf.buffer().end() - buf.position();
|
||||
|
||||
s.append(buf.position(), bytes);
|
||||
buf.position() += bytes;
|
||||
|
||||
if (buf.hasPendingData())
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
/** Позволяет найти в куске памяти следующий символ \t, \n или \\.
|
||||
* Функция похожа на strpbrk, но со следующими отличиями:
|
||||
@ -244,21 +265,28 @@ void readBackQuotedString(String & s, ReadBuffer & buf)
|
||||
|
||||
void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf)
|
||||
{
|
||||
char s[19];
|
||||
static constexpr auto DATE_TIME_BROKEN_DOWN_LENGTH = 19;
|
||||
static constexpr auto UNIX_TIMESTAMP_MAX_LENGTH = 10;
|
||||
|
||||
size_t size = buf.read(s, 10);
|
||||
if (10 != size)
|
||||
char s[DATE_TIME_BROKEN_DOWN_LENGTH];
|
||||
char * s_pos = s;
|
||||
|
||||
/// Кусок, похожий на unix timestamp.
|
||||
while (s_pos < s + UNIX_TIMESTAMP_MAX_LENGTH && !buf.eof() && *buf.position() >= '0' && *buf.position() <= '9')
|
||||
{
|
||||
s[size] = 0;
|
||||
throw Exception(std::string("Cannot parse datetime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME);
|
||||
*s_pos = *buf.position();
|
||||
++s_pos;
|
||||
++buf.position();
|
||||
}
|
||||
|
||||
if (s[4] < '0' || s[4] > '9')
|
||||
/// 2015-01-01 01:02:03
|
||||
if (s_pos == s + 4 && !buf.eof() && (*buf.position() < '0' || *buf.position() > '9'))
|
||||
{
|
||||
size_t size = buf.read(&s[10], 9);
|
||||
if (9 != size)
|
||||
const size_t remaining_size = DATE_TIME_BROKEN_DOWN_LENGTH - (s_pos - s);
|
||||
size_t size = buf.read(s_pos, remaining_size);
|
||||
if (remaining_size != size)
|
||||
{
|
||||
s[10 + size] = 0;
|
||||
s_pos[size] = 0;
|
||||
throw Exception(std::string("Cannot parse datetime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME);
|
||||
}
|
||||
|
||||
@ -276,7 +304,7 @@ void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf)
|
||||
datetime = DateLUT::instance().makeDateTime(year, month, day, hour, minute, second);
|
||||
}
|
||||
else
|
||||
datetime = parse<time_t>(s, 10);
|
||||
datetime = parse<time_t>(s, s_pos - s);
|
||||
}
|
||||
|
||||
|
||||
|
1000000
dbms/src/IO/tests/DevicePixelRatio
Normal file
1000000
dbms/src/IO/tests/DevicePixelRatio
Normal file
File diff suppressed because it is too large
Load Diff
40
dbms/src/IO/tests/read_float_perf.cpp
Normal file
40
dbms/src/IO/tests/read_float_perf.cpp
Normal file
@ -0,0 +1,40 @@
|
||||
#include <string>
|
||||
|
||||
#include <iostream>
|
||||
#include <fstream>
|
||||
|
||||
#include <DB/Core/Types.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
#include <DB/IO/ReadBufferFromIStream.h>
|
||||
#include <DB/IO/CompressedReadBuffer.h>
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
try
|
||||
{
|
||||
std::ifstream istr("DevicePixelRatio");
|
||||
DB::ReadBufferFromIStream in(istr);
|
||||
|
||||
DB::Float32 b = 0;
|
||||
|
||||
size_t i = 0;
|
||||
while (!in.eof())
|
||||
{
|
||||
DB::readFloatText(b, in);
|
||||
in.ignore();
|
||||
|
||||
++i;
|
||||
}
|
||||
|
||||
std::cout << b << std::endl;
|
||||
std::cout << i << std::endl;
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
return 1;
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
@ -715,6 +715,15 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
result.convertToTwoLevel();
|
||||
|
||||
/// Проверка ограничений.
|
||||
if (!checkLimits(result_size, no_more_keys))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const
|
||||
{
|
||||
if (!no_more_keys && max_rows_to_group_by && result_size > max_rows_to_group_by)
|
||||
{
|
||||
if (group_by_overflow_mode == OverflowMode::THROW)
|
||||
@ -1226,6 +1235,86 @@ void NO_INLINE Aggregator::mergeDataImpl(
|
||||
}
|
||||
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl(
|
||||
Table & table_dst,
|
||||
AggregatedDataWithoutKey & overflows,
|
||||
Table & table_src) const
|
||||
{
|
||||
for (auto it = table_src.begin(); it != table_src.end(); ++it)
|
||||
{
|
||||
decltype(it) res_it = table_dst.find(it->first, it.getHash());
|
||||
|
||||
AggregateDataPtr res_data = table_dst.end() == res_it
|
||||
? overflows
|
||||
: Method::getAggregateData(res_it->second);
|
||||
|
||||
for (size_t i = 0; i < aggregates_size; ++i)
|
||||
aggregate_functions[i]->merge(
|
||||
res_data + offsets_of_aggregate_states[i],
|
||||
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
||||
|
||||
for (size_t i = 0; i < aggregates_size; ++i)
|
||||
aggregate_functions[i]->destroy(
|
||||
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
||||
|
||||
Method::getAggregateData(it->second) = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl(
|
||||
Table & table_dst,
|
||||
Table & table_src) const
|
||||
{
|
||||
for (auto it = table_src.begin(); it != table_src.end(); ++it)
|
||||
{
|
||||
decltype(it) res_it = table_dst.find(it->first, it.getHash());
|
||||
|
||||
if (table_dst.end() == res_it)
|
||||
continue;
|
||||
|
||||
AggregateDataPtr res_data = Method::getAggregateData(res_it->second);
|
||||
|
||||
for (size_t i = 0; i < aggregates_size; ++i)
|
||||
aggregate_functions[i]->merge(
|
||||
res_data + offsets_of_aggregate_states[i],
|
||||
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
||||
|
||||
for (size_t i = 0; i < aggregates_size; ++i)
|
||||
aggregate_functions[i]->destroy(
|
||||
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
||||
|
||||
Method::getAggregateData(it->second) = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void NO_INLINE Aggregator::mergeDataRemainingKeysToOverflowsImpl(
|
||||
AggregatedDataWithoutKey & overflows,
|
||||
Table & table_src) const
|
||||
{
|
||||
for (auto it = table_src.begin(); it != table_src.end(); ++it)
|
||||
{
|
||||
if (Method::getAggregateData(it->second) == nullptr)
|
||||
continue;
|
||||
|
||||
AggregateDataPtr res_data = overflows;
|
||||
|
||||
for (size_t i = 0; i < aggregates_size; ++i)
|
||||
aggregate_functions[i]->merge(
|
||||
res_data + offsets_of_aggregate_states[i],
|
||||
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
||||
|
||||
for (size_t i = 0; i < aggregates_size; ++i)
|
||||
aggregate_functions[i]->destroy(
|
||||
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
||||
|
||||
Method::getAggregateData(it->second) = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void NO_INLINE Aggregator::mergeWithoutKeyDataImpl(
|
||||
ManyAggregatedDataVariants & non_empty_data) const
|
||||
{
|
||||
@ -1253,15 +1342,25 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl(
|
||||
ManyAggregatedDataVariants & non_empty_data) const
|
||||
{
|
||||
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
||||
bool no_more_keys = false;
|
||||
|
||||
/// Все результаты агрегации соединяем с первым.
|
||||
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
||||
{
|
||||
if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys))
|
||||
break;
|
||||
|
||||
AggregatedDataVariants & current = *non_empty_data[i];
|
||||
|
||||
mergeDataImpl<Method>(
|
||||
getDataVariant<Method>(*res).data,
|
||||
getDataVariant<Method>(current).data);
|
||||
if (!no_more_keys)
|
||||
mergeDataImpl<Method>(
|
||||
getDataVariant<Method>(*res).data,
|
||||
getDataVariant<Method>(current).data);
|
||||
else
|
||||
mergeDataNoMoreKeysImpl<Method>(
|
||||
getDataVariant<Method>(*res).data,
|
||||
res->without_key,
|
||||
getDataVariant<Method>(current).data);
|
||||
|
||||
/// current не будет уничтожать состояния агрегатных функций в деструкторе
|
||||
current.aggregator = nullptr;
|
||||
@ -1276,8 +1375,13 @@ void NO_INLINE Aggregator::mergeTwoLevelDataImpl(
|
||||
{
|
||||
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
||||
|
||||
/// В данном случае, no_more_keys будет выставлено, только если в первом (самом большом) состоянии достаточно много строк.
|
||||
bool no_more_keys = false;
|
||||
if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys))
|
||||
return;
|
||||
|
||||
/// Слияние распараллеливается по корзинам - первому уровню TwoLevelHashMap.
|
||||
auto merge_bucket = [&non_empty_data, &res, this](size_t bucket, MemoryTracker * memory_tracker)
|
||||
auto merge_bucket = [&non_empty_data, &res, no_more_keys, this](size_t bucket, MemoryTracker * memory_tracker)
|
||||
{
|
||||
current_memory_tracker = memory_tracker;
|
||||
|
||||
@ -1286,12 +1390,18 @@ void NO_INLINE Aggregator::mergeTwoLevelDataImpl(
|
||||
{
|
||||
AggregatedDataVariants & current = *non_empty_data[i];
|
||||
|
||||
mergeDataImpl<Method>(
|
||||
getDataVariant<Method>(*res).data.impls[bucket],
|
||||
getDataVariant<Method>(current).data.impls[bucket]);
|
||||
|
||||
/// current не будет уничтожать состояния агрегатных функций в деструкторе
|
||||
current.aggregator = nullptr;
|
||||
if (!no_more_keys)
|
||||
{
|
||||
mergeDataImpl<Method>(
|
||||
getDataVariant<Method>(*res).data.impls[bucket],
|
||||
getDataVariant<Method>(current).data.impls[bucket]);
|
||||
}
|
||||
else
|
||||
{
|
||||
mergeDataOnlyExistingKeysImpl<Method>(
|
||||
getDataVariant<Method>(*res).data.impls[bucket],
|
||||
getDataVariant<Method>(current).data.impls[bucket]);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
@ -1326,6 +1436,25 @@ void NO_INLINE Aggregator::mergeTwoLevelDataImpl(
|
||||
for (auto & task : tasks)
|
||||
if (task.valid())
|
||||
task.get_future().get();
|
||||
|
||||
if (no_more_keys && overflow_row)
|
||||
{
|
||||
for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket)
|
||||
{
|
||||
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
||||
{
|
||||
AggregatedDataVariants & current = *non_empty_data[i];
|
||||
|
||||
mergeDataRemainingKeysToOverflowsImpl<Method>(
|
||||
res->without_key,
|
||||
getDataVariant<Method>(current).data.impls[bucket]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// aggregator не будет уничтожать состояния агрегатных функций в деструкторе
|
||||
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
||||
non_empty_data[i]->aggregator = nullptr;
|
||||
}
|
||||
|
||||
|
||||
@ -1350,6 +1479,13 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va
|
||||
if (non_empty_data.size() == 1)
|
||||
return non_empty_data[0];
|
||||
|
||||
/// Отсортируем состояния по убыванию размера, чтобы мердж был более эффективным (так как все состояния мерджатся в первое).
|
||||
std::sort(non_empty_data.begin(), non_empty_data.end(),
|
||||
[](const AggregatedDataVariantsPtr & lhs, const AggregatedDataVariantsPtr & rhs)
|
||||
{
|
||||
return lhs->sizeWithoutOverflowRow() > rhs->sizeWithoutOverflowRow();
|
||||
});
|
||||
|
||||
/// Если хотя бы один из вариантов двухуровневый, то переконвертируем все варианты в двухуровневые, если есть не такие.
|
||||
/// Замечание - возможно, было бы более оптимально не конвертировать одноуровневые варианты перед мерджем, а мерджить их отдельно, в конце.
|
||||
|
||||
@ -1448,13 +1584,14 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va
|
||||
}
|
||||
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void NO_INLINE Aggregator::mergeStreamsImpl(
|
||||
template <bool no_more_keys, typename Method, typename Table>
|
||||
void NO_INLINE Aggregator::mergeStreamsImplCase(
|
||||
Block & block,
|
||||
const Sizes & key_sizes,
|
||||
Arena * aggregates_pool,
|
||||
Method & method,
|
||||
Table & data) const
|
||||
Table & data,
|
||||
AggregateDataPtr overflow_row) const
|
||||
{
|
||||
ConstColumnPlainPtrs key_columns(keys_size);
|
||||
AggregateColumnsData aggregate_columns(aggregates_size);
|
||||
@ -1475,13 +1612,33 @@ void NO_INLINE Aggregator::mergeStreamsImpl(
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
typename Table::iterator it;
|
||||
|
||||
bool inserted; /// Вставили новый ключ, или такой ключ уже был?
|
||||
bool overflow = false; /// Новый ключ не поместился в хэш-таблицу из-за no_more_keys.
|
||||
|
||||
/// Получаем ключ для вставки в хэш-таблицу.
|
||||
auto key = state.getKey(key_columns, keys_size, i, key_sizes, keys, *aggregates_pool);
|
||||
|
||||
data.emplace(key, it, inserted);
|
||||
if (!no_more_keys)
|
||||
{
|
||||
data.emplace(key, it, inserted);
|
||||
}
|
||||
else
|
||||
{
|
||||
inserted = false;
|
||||
it = data.find(key);
|
||||
if (data.end() == it)
|
||||
overflow = true;
|
||||
}
|
||||
|
||||
/// Если ключ не поместился, и данные не надо агрегировать в отдельную строку, то делать нечего.
|
||||
if (no_more_keys && overflow && !overflow_row)
|
||||
{
|
||||
method.onExistingKey(key, keys, *aggregates_pool);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
|
||||
if (inserted)
|
||||
{
|
||||
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
|
||||
@ -1496,10 +1653,12 @@ void NO_INLINE Aggregator::mergeStreamsImpl(
|
||||
else
|
||||
method.onExistingKey(key, keys, *aggregates_pool);
|
||||
|
||||
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
|
||||
|
||||
/// Мерджим состояния агрегатных функций.
|
||||
for (size_t j = 0; j < aggregates_size; ++j)
|
||||
aggregate_functions[j]->merge(
|
||||
Method::getAggregateData(it->second) + offsets_of_aggregate_states[j],
|
||||
value + offsets_of_aggregate_states[j],
|
||||
(*aggregate_columns[j])[i]);
|
||||
}
|
||||
|
||||
@ -1507,6 +1666,23 @@ void NO_INLINE Aggregator::mergeStreamsImpl(
|
||||
block.clear();
|
||||
}
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void NO_INLINE Aggregator::mergeStreamsImpl(
|
||||
Block & block,
|
||||
const Sizes & key_sizes,
|
||||
Arena * aggregates_pool,
|
||||
Method & method,
|
||||
Table & data,
|
||||
AggregateDataPtr overflow_row,
|
||||
bool no_more_keys) const
|
||||
{
|
||||
if (!no_more_keys)
|
||||
mergeStreamsImplCase<false>(block, key_sizes, aggregates_pool, method, data, overflow_row);
|
||||
else
|
||||
mergeStreamsImplCase<true>(block, key_sizes, aggregates_pool, method, data, overflow_row);
|
||||
}
|
||||
|
||||
|
||||
void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
|
||||
Block & block,
|
||||
AggregatedDataVariants & result) const
|
||||
@ -1621,6 +1797,11 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
/// Сначала параллельно мерджим для отдельных bucket-ов. Затем домердживаем данные, не распределённые по bucket-ам.
|
||||
if (has_two_level)
|
||||
{
|
||||
/** В этом случае, no_more_keys не поддерживается в связи с тем, что
|
||||
* из разных потоков трудно обновлять общее состояние для "остальных" ключей (overflows).
|
||||
* То есть, ключей в итоге может оказаться существенно больше, чем max_rows_to_group_by.
|
||||
*/
|
||||
|
||||
LOG_TRACE(log, "Merging partially aggregated two-level data.");
|
||||
|
||||
auto merge_bucket = [&bucket_to_blocks, &result, &key_sizes, this](Int32 bucket, Arena * aggregates_pool, MemoryTracker * memory_tracker)
|
||||
@ -1634,7 +1815,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
|
||||
#define M(NAME) \
|
||||
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
||||
mergeStreamsImpl(block, key_sizes, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket]);
|
||||
mergeStreamsImpl(block, key_sizes, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket], nullptr, false);
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
||||
@ -1691,6 +1872,8 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
{
|
||||
LOG_TRACE(log, "Merging partially aggregated single-level data.");
|
||||
|
||||
bool no_more_keys = false;
|
||||
|
||||
BlocksList & blocks = bucket_to_blocks[-1];
|
||||
for (Block & block : blocks)
|
||||
{
|
||||
@ -1700,12 +1883,15 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
return;
|
||||
}
|
||||
|
||||
if (!checkLimits(result.sizeWithoutOverflowRow(), no_more_keys))
|
||||
break;
|
||||
|
||||
if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows)
|
||||
mergeWithoutKeyStreamsImpl(block, result);
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
||||
mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data);
|
||||
mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys);
|
||||
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
#undef M
|
||||
@ -1760,7 +1946,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
||||
mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data);
|
||||
mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, false);
|
||||
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
#undef M
|
||||
|
@ -1,56 +1,104 @@
|
||||
#include <DB/Interpreters/Cluster.h>
|
||||
#include <DB/Common/escapeForFileName.h>
|
||||
#include <DB/Common/isLocalAddress.h>
|
||||
#include <DB/Common/SimpleCache.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Вес шарда по-умолчанию.
|
||||
static constexpr int default_weight = 1;
|
||||
|
||||
inline bool isLocal(const Cluster::Address & address)
|
||||
{
|
||||
/// Если среди реплик существует такая, что:
|
||||
/// - её порт совпадает с портом, который слушает сервер;
|
||||
/// - её хост резолвится в набор адресов, один из которых совпадает с одним из адресов сетевых интерфейсов сервера
|
||||
/// то нужно всегда ходить на этот шард без межпроцессного взаимодействия
|
||||
return isLocalAddress(address.resolved_address);
|
||||
}
|
||||
|
||||
inline std::string addressToDirName(const Cluster::Address & address)
|
||||
{
|
||||
return
|
||||
escapeForFileName(address.user) +
|
||||
(address.password.empty() ? "" : (':' + escapeForFileName(address.password))) + '@' +
|
||||
escapeForFileName(address.resolved_address.host().toString()) + ':' +
|
||||
std::to_string(address.resolved_address.port());
|
||||
}
|
||||
|
||||
inline bool beginsWith(const std::string & str1, const char * str2)
|
||||
{
|
||||
if (str2 == nullptr)
|
||||
throw Exception("Passed null pointer to function beginsWith", ErrorCodes::LOGICAL_ERROR);
|
||||
return 0 == strncmp(str1.data(), str2, strlen(str2));
|
||||
}
|
||||
|
||||
/// Для кэширования DNS запросов.
|
||||
Poco::Net::SocketAddress resolveSocketAddressImpl1(const String & host, UInt16 port)
|
||||
{
|
||||
return Poco::Net::SocketAddress(host, port);
|
||||
}
|
||||
|
||||
Poco::Net::SocketAddress resolveSocketAddressImpl2(const String & host_and_port)
|
||||
{
|
||||
return Poco::Net::SocketAddress(host_and_port);
|
||||
}
|
||||
|
||||
Poco::Net::SocketAddress resolveSocketAddress(const String & host, UInt16 port)
|
||||
{
|
||||
static SimpleCache<decltype(resolveSocketAddressImpl1), &resolveSocketAddressImpl1> cache;
|
||||
return cache(host, port);
|
||||
}
|
||||
|
||||
Poco::Net::SocketAddress resolveSocketAddress(const String & host_and_port)
|
||||
{
|
||||
static SimpleCache<decltype(resolveSocketAddressImpl2), &resolveSocketAddressImpl2> cache;
|
||||
return cache(host_and_port);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/// Реализация класса Cluster::Address
|
||||
|
||||
Cluster::Address::Address(const String & config_prefix)
|
||||
{
|
||||
auto & config = Poco::Util::Application::instance().config();
|
||||
const auto & config = Poco::Util::Application::instance().config();
|
||||
|
||||
host_name = config.getString(config_prefix + ".host");
|
||||
port = config.getInt(config_prefix + ".port");
|
||||
resolved_address = Poco::Net::SocketAddress(host_name, port);
|
||||
resolved_address = resolveSocketAddress(host_name, port);
|
||||
user = config.getString(config_prefix + ".user", "default");
|
||||
password = config.getString(config_prefix + ".password", "");
|
||||
}
|
||||
|
||||
|
||||
Cluster::Address::Address(const String & host_port_, const String & user_, const String & password_)
|
||||
: user(user_), password(password_)
|
||||
{
|
||||
UInt16 default_port = Poco::Util::Application::instance().config().getInt("tcp_port", 0);
|
||||
|
||||
/// Похоже на то, что строка host_port_ содержит порт. Если условие срабатывает - не обязательно значит, что порт есть (пример: [::]).
|
||||
if (nullptr != strchr(host_port_.c_str(), ':') || !default_port)
|
||||
if ((nullptr != strchr(host_port_.c_str(), ':')) || !default_port)
|
||||
{
|
||||
resolved_address = Poco::Net::SocketAddress(host_port_);
|
||||
resolved_address = resolveSocketAddress(host_port_);
|
||||
host_name = host_port_.substr(0, host_port_.find(':'));
|
||||
port = resolved_address.port();
|
||||
}
|
||||
else
|
||||
{
|
||||
resolved_address = Poco::Net::SocketAddress(host_port_, default_port);
|
||||
resolved_address = resolveSocketAddress(host_port_, default_port);
|
||||
host_name = host_port_;
|
||||
port = default_port;
|
||||
}
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
inline std::string addressToDirName(const Cluster::Address & address)
|
||||
{
|
||||
return
|
||||
escapeForFileName(address.user) +
|
||||
(address.password.empty() ? "" : (':' + escapeForFileName(address.password))) + '@' +
|
||||
escapeForFileName(address.resolved_address.host().toString()) + ':' +
|
||||
std::to_string(address.resolved_address.port());
|
||||
}
|
||||
}
|
||||
|
||||
/// Реализация класса Clusters
|
||||
|
||||
Clusters::Clusters(const Settings & settings, const String & config_name)
|
||||
{
|
||||
@ -58,12 +106,13 @@ Clusters::Clusters(const Settings & settings, const String & config_name)
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys(config_name, config_keys);
|
||||
|
||||
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it)
|
||||
for (const auto & key : config_keys)
|
||||
impl.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(*it),
|
||||
std::forward_as_tuple(settings, config_name + "." + *it));
|
||||
std::forward_as_tuple(key),
|
||||
std::forward_as_tuple(settings, config_name + "." + key));
|
||||
}
|
||||
|
||||
/// Реализация класса Cluster
|
||||
|
||||
Cluster::Cluster(const Settings & settings, const String & cluster_name)
|
||||
{
|
||||
@ -75,35 +124,56 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name)
|
||||
|
||||
UInt32 current_shard_num = 1;
|
||||
|
||||
for (auto it = config_keys.begin(); it != config_keys.end(); ++it)
|
||||
for (const auto & key : config_keys)
|
||||
{
|
||||
if (0 == strncmp(it->c_str(), "node", strlen("node")))
|
||||
if (beginsWith(key, "node"))
|
||||
{
|
||||
const auto & prefix = config_prefix + *it;
|
||||
const auto weight = config.getInt(prefix + ".weight", 1);
|
||||
/// Шард без реплик.
|
||||
|
||||
const auto & prefix = config_prefix + key;
|
||||
const auto weight = config.getInt(prefix + ".weight", default_weight);
|
||||
if (weight == 0)
|
||||
continue;
|
||||
|
||||
addresses.emplace_back(prefix);
|
||||
addresses.back().replica_num = 1;
|
||||
const auto & address = addresses.back();
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
|
||||
if (const auto is_local = isLocal(addresses.back()))
|
||||
shard_info_vec.push_back({{}, current_shard_num, weight, is_local});
|
||||
ShardInfo info;
|
||||
info.shard_num = current_shard_num;
|
||||
info.weight = weight;
|
||||
|
||||
if (isLocal(address))
|
||||
info.local_addresses.push_back(address);
|
||||
else
|
||||
shard_info_vec.push_back({{addressToDirName(addresses.back())}, current_shard_num, weight, is_local});
|
||||
{
|
||||
info.dir_names.push_back(addressToDirName(address));
|
||||
info.pool = new ConnectionPool(
|
||||
settings.distributed_connections_pool_size,
|
||||
address.host_name, address.port, address.resolved_address,
|
||||
"", address.user, address.password,
|
||||
"server", Protocol::Compression::Enable,
|
||||
saturate(settings.connect_timeout, settings.limits.max_execution_time),
|
||||
saturate(settings.receive_timeout, settings.limits.max_execution_time),
|
||||
saturate(settings.send_timeout, settings.limits.max_execution_time));
|
||||
}
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size());
|
||||
shards_info.push_back(info);
|
||||
}
|
||||
else if (0 == strncmp(it->c_str(), "shard", strlen("shard")))
|
||||
else if (beginsWith(key, "shard"))
|
||||
{
|
||||
/// Шард с репликами.
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys replica_keys;
|
||||
config.keys(config_prefix + *it, replica_keys);
|
||||
config.keys(config_prefix + key, replica_keys);
|
||||
|
||||
addresses_with_failover.emplace_back();
|
||||
Addresses & replica_addresses = addresses_with_failover.back();
|
||||
UInt32 current_replica_num = 1;
|
||||
|
||||
const auto & partial_prefix = config_prefix + *it + ".";
|
||||
const auto weight = config.getInt(partial_prefix + ".weight", 1);
|
||||
const auto & partial_prefix = config_prefix + key + ".";
|
||||
const auto weight = config.getInt(partial_prefix + ".weight", default_weight);
|
||||
if (weight == 0)
|
||||
continue;
|
||||
|
||||
@ -113,26 +183,20 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name)
|
||||
* the first element of vector; otherwise we will just .emplace_back
|
||||
*/
|
||||
std::vector<std::string> dir_names{};
|
||||
size_t num_local_nodes = 0;
|
||||
|
||||
auto first = true;
|
||||
for (auto jt = replica_keys.begin(); jt != replica_keys.end(); ++jt)
|
||||
for (const auto & replica_key : replica_keys)
|
||||
{
|
||||
if (0 == strncmp(jt->data(), "weight", strlen("weight")) ||
|
||||
0 == strncmp(jt->data(), "internal_replication", strlen("internal_replication")))
|
||||
if (beginsWith(replica_key, "weight") || beginsWith(replica_key, "internal_replication"))
|
||||
continue;
|
||||
|
||||
if (0 == strncmp(jt->c_str(), "replica", strlen("replica")))
|
||||
if (beginsWith(replica_key, "replica"))
|
||||
{
|
||||
replica_addresses.emplace_back(partial_prefix + *jt);
|
||||
replica_addresses.emplace_back(partial_prefix + replica_key);
|
||||
replica_addresses.back().replica_num = current_replica_num;
|
||||
++current_replica_num;
|
||||
|
||||
if (isLocal(replica_addresses.back()))
|
||||
{
|
||||
++num_local_nodes;
|
||||
}
|
||||
else
|
||||
if (!isLocal(replica_addresses.back()))
|
||||
{
|
||||
if (internal_replication)
|
||||
{
|
||||
@ -149,36 +213,18 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name)
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown element in config: " + *jt, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
throw Exception("Unknown element in config: " + replica_key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
}
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
|
||||
shard_info_vec.push_back({std::move(dir_names), current_shard_num, weight, num_local_nodes});
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
Addresses shard_local_addresses;
|
||||
|
||||
++current_shard_num;
|
||||
}
|
||||
|
||||
if (!addresses_with_failover.empty() && !addresses.empty())
|
||||
throw Exception("There must be either 'node' or 'shard' elements in config", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
if (addresses_with_failover.size())
|
||||
{
|
||||
for (const auto & shard : addresses_with_failover)
|
||||
{
|
||||
ConnectionPools replicas;
|
||||
replicas.reserve(shard.size());
|
||||
replicas.reserve(replica_addresses.size());
|
||||
|
||||
bool has_local_replics = false;
|
||||
for (const auto & replica : shard)
|
||||
for (const auto & replica : replica_addresses)
|
||||
{
|
||||
if (isLocal(replica))
|
||||
{
|
||||
has_local_replics = true;
|
||||
break;
|
||||
}
|
||||
shard_local_addresses.push_back(replica);
|
||||
else
|
||||
{
|
||||
replicas.emplace_back(new ConnectionPool(
|
||||
@ -192,55 +238,43 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name)
|
||||
}
|
||||
}
|
||||
|
||||
if (has_local_replics)
|
||||
++local_nodes_num;
|
||||
else
|
||||
pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
|
||||
ConnectionPoolPtr shard_pool;
|
||||
if (!replicas.empty())
|
||||
shard_pool = new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size());
|
||||
shards_info.push_back({std::move(dir_names), current_shard_num, weight, shard_local_addresses, shard_pool});
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
if (!addresses_with_failover.empty() && !addresses.empty())
|
||||
throw Exception("There must be either 'node' or 'shard' elements in config", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
++current_shard_num;
|
||||
}
|
||||
else if (addresses.size())
|
||||
{
|
||||
for (const auto & address : addresses)
|
||||
{
|
||||
if (isLocal(address))
|
||||
{
|
||||
++local_nodes_num;
|
||||
}
|
||||
else
|
||||
{
|
||||
pools.emplace_back(new ConnectionPool(
|
||||
settings.distributed_connections_pool_size,
|
||||
address.host_name, address.port, address.resolved_address,
|
||||
"", address.user, address.password,
|
||||
"server", Protocol::Compression::Enable,
|
||||
saturate(settings.connect_timeout, settings.limits.max_execution_time),
|
||||
saturate(settings.receive_timeout, settings.limits.max_execution_time),
|
||||
saturate(settings.send_timeout, settings.limits.max_execution_time)));
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("No addresses listed in config", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
||||
|
||||
initMisc();
|
||||
}
|
||||
|
||||
|
||||
Cluster::Cluster(const Settings & settings, std::vector<std::vector<String>> names,
|
||||
const String & username, const String & password)
|
||||
{
|
||||
UInt32 current_shard_num = 1;
|
||||
|
||||
for (const auto & shard : names)
|
||||
{
|
||||
Addresses current;
|
||||
for (auto & replica : shard)
|
||||
current.emplace_back(replica, username, password);
|
||||
|
||||
addresses_with_failover.emplace_back(current);
|
||||
}
|
||||
|
||||
for (const auto & shard : addresses_with_failover)
|
||||
{
|
||||
ConnectionPools replicas;
|
||||
replicas.reserve(shard.size());
|
||||
replicas.reserve(current.size());
|
||||
|
||||
for (const auto & replica : shard)
|
||||
for (const auto & replica : current)
|
||||
{
|
||||
replicas.emplace_back(new ConnectionPool(
|
||||
settings.distributed_connections_pool_size,
|
||||
@ -252,8 +286,14 @@ Cluster::Cluster(const Settings & settings, std::vector<std::vector<String>> nam
|
||||
saturate(settings.send_timeout, settings.limits.max_execution_time)));
|
||||
}
|
||||
|
||||
pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
|
||||
ConnectionPoolPtr shard_pool = new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), default_weight, shards_info.size());
|
||||
shards_info.push_back({{}, current_shard_num, default_weight, {}, shard_pool});
|
||||
++current_shard_num;
|
||||
}
|
||||
|
||||
initMisc();
|
||||
}
|
||||
|
||||
|
||||
@ -262,17 +302,35 @@ Poco::Timespan Cluster::saturate(const Poco::Timespan & v, const Poco::Timespan
|
||||
if (limit.totalMicroseconds() == 0)
|
||||
return v;
|
||||
else
|
||||
return v > limit ? limit : v;
|
||||
return (v > limit) ? limit : v;
|
||||
}
|
||||
|
||||
|
||||
bool Cluster::isLocal(const Address & address)
|
||||
void Cluster::initMisc()
|
||||
{
|
||||
/// Если среди реплик существует такая, что:
|
||||
/// - её порт совпадает с портом, который слушает сервер;
|
||||
/// - её хост резолвится в набор адресов, один из которых совпадает с одним из адресов сетевых интерфейсов сервера
|
||||
/// то нужно всегда ходить на этот шард без межпроцессного взаимодействия
|
||||
return isLocalAddress(address.resolved_address);
|
||||
for (const auto & shard_info : shards_info)
|
||||
{
|
||||
if (!shard_info.isLocal() && !shard_info.hasRemoteConnections())
|
||||
throw Exception("Found shard without any specified connection",
|
||||
ErrorCodes::SHARD_HAS_NO_CONNECTIONS);
|
||||
}
|
||||
|
||||
for (const auto & shard_info : shards_info)
|
||||
{
|
||||
if (shard_info.isLocal())
|
||||
++local_shard_count;
|
||||
else
|
||||
++remote_shard_count;
|
||||
}
|
||||
|
||||
for (auto & shard_info : shards_info)
|
||||
{
|
||||
if (!shard_info.isLocal())
|
||||
{
|
||||
any_remote_shard_info = &shard_info;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -148,9 +148,9 @@ void ExpressionAction::prepare(Block & sample_block)
|
||||
|
||||
case ARRAY_JOIN:
|
||||
{
|
||||
for (NameSet::iterator it = array_joined_columns.begin(); it != array_joined_columns.end(); ++it)
|
||||
for (const auto & name : array_joined_columns)
|
||||
{
|
||||
ColumnWithTypeAndName & current = sample_block.getByName(*it);
|
||||
ColumnWithTypeAndName & current = sample_block.getByName(name);
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*current.type);
|
||||
if (!array_type)
|
||||
throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH);
|
||||
@ -214,6 +214,7 @@ void ExpressionAction::prepare(Block & sample_block)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAction::execute(Block & block) const
|
||||
{
|
||||
// std::cerr << "executing: " << toString() << std::endl;
|
||||
@ -261,9 +262,11 @@ void ExpressionAction::execute(Block & block) const
|
||||
{
|
||||
if (array_joined_columns.empty())
|
||||
throw Exception("No arrays to join", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
ColumnPtr any_array_ptr = block.getByName(*array_joined_columns.begin()).column;
|
||||
if (any_array_ptr->isConst())
|
||||
any_array_ptr = dynamic_cast<const IColumnConst &>(*any_array_ptr).convertToFullColumn();
|
||||
|
||||
const ColumnArray * any_array = typeid_cast<const ColumnArray *>(&*any_array_ptr);
|
||||
if (!any_array)
|
||||
throw Exception("ARRAY JOIN of not array: " + *array_joined_columns.begin(), ErrorCodes::TYPE_MISMATCH);
|
||||
@ -552,19 +555,22 @@ bool ExpressionActions::popUnusedArrayJoin(const Names & required_columns, Expre
|
||||
{
|
||||
if (actions.empty() || actions.back().type != ExpressionAction::ARRAY_JOIN)
|
||||
return false;
|
||||
|
||||
NameSet required_set(required_columns.begin(), required_columns.end());
|
||||
|
||||
for (const std::string & name : actions.back().array_joined_columns)
|
||||
{
|
||||
if (required_set.count(name))
|
||||
return false;
|
||||
}
|
||||
|
||||
for (const std::string & name : actions.back().array_joined_columns)
|
||||
{
|
||||
DataTypePtr & type = sample_block.getByName(name).type;
|
||||
type = new DataTypeArray(type);
|
||||
}
|
||||
|
||||
out_action = actions.back();
|
||||
actions.pop_back();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -781,6 +787,9 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
|
||||
for (const auto & name : action.prerequisite_names)
|
||||
++columns_refcount[name];
|
||||
|
||||
for (const auto & name_alias : action.projection)
|
||||
++columns_refcount[name_alias.first];
|
||||
}
|
||||
|
||||
Actions new_actions;
|
||||
@ -809,6 +818,8 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
|
||||
for (const auto & name : action.prerequisite_names)
|
||||
process(name);
|
||||
|
||||
/// Для projection тут нет уменьшения refcount, так как действие project заменяет имена у столбцов, по сути, уже удаляя их под старыми именами.
|
||||
}
|
||||
|
||||
actions.swap(new_actions);
|
||||
@ -1010,7 +1021,11 @@ void ExpressionActionsChain::finalize()
|
||||
steps[i].actions->finalize(required_output);
|
||||
}
|
||||
|
||||
/// Когда возможно, перенесем ARRAY JOIN из более ранних шагов в более поздние.
|
||||
/** Когда возможно, перенесем ARRAY JOIN из более ранних шагов в более поздние.
|
||||
* Замечание: обычно это полезно, так как ARRAY JOIN - сложная операция, которая, как правило, увеличивает объём данных.
|
||||
* Но не всегда - в случае, если большинство массивов пустые, ARRAY JOIN, наоборот, уменьшает объём данных,
|
||||
* и его было бы полезно делать раньше. Этот случай не рассматривается.
|
||||
*/
|
||||
for (size_t i = 1; i < steps.size(); ++i)
|
||||
{
|
||||
ExpressionAction action;
|
||||
|
@ -71,6 +71,7 @@ const std::unordered_set<String> injective_function_names
|
||||
"bitmaskToArray",
|
||||
"tuple",
|
||||
"regionToName",
|
||||
"concatAssumeInjective",
|
||||
};
|
||||
|
||||
const std::unordered_set<String> possibly_injective_function_names
|
||||
@ -140,12 +141,18 @@ void ExpressionAnalyzer::init()
|
||||
/// Удалить ненужное из списка columns. Создать unknown_required_columns. Сформировать columns_added_by_join.
|
||||
collectUsedColumns();
|
||||
|
||||
/// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns.
|
||||
analyzeAggregation();
|
||||
|
||||
/// external_tables, subqueries_for_sets для глобальных подзапросов.
|
||||
/// Заменяет глобальные подзапросы на сгенерированные имена временных таблиц, которые будут отправлены на удалённые серверы.
|
||||
initGlobalSubqueriesAndExternalTables();
|
||||
|
||||
/// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns.
|
||||
/// Этот анализ надо провести после обработки глобальных подзапросов, потому что в противном случае,
|
||||
/// если агрегатная функция содержит глобальный подзапрос, то метод analyzeAggregation сохранит
|
||||
/// в aggregate_descriptions информацию о параметрах этой агрегатной функции, среди которых окажется
|
||||
/// глобальный подзапрос. Затем при вызове метода initGlobalSubqueriesAndExternalTables, этот
|
||||
/// глобальный подзапрос будет заменён на временную таблицу, в результате чего aggregate_descriptions
|
||||
/// будет содержать устаревшую информацию, что приведёт к ошибке при выполнении запроса.
|
||||
analyzeAggregation();
|
||||
}
|
||||
|
||||
|
||||
@ -204,13 +211,17 @@ void ExpressionAnalyzer::analyzeAggregation()
|
||||
/// constant expressions have non-null column pointer at this stage
|
||||
if (const auto is_constexpr = col.column)
|
||||
{
|
||||
if (i < group_asts.size() - 1)
|
||||
group_asts[i] = std::move(group_asts.back());
|
||||
/// but don't remove last key column if no aggregate functions, otherwise aggregation will not work
|
||||
if (!aggregate_descriptions.empty() || group_asts.size() > 1)
|
||||
{
|
||||
if (i < group_asts.size() - 1)
|
||||
group_asts[i] = std::move(group_asts.back());
|
||||
|
||||
group_asts.pop_back();
|
||||
i -= 1;
|
||||
group_asts.pop_back();
|
||||
i -= 1;
|
||||
|
||||
continue;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
NameAndTypePair key{column_name, col.type};
|
||||
@ -774,7 +785,26 @@ void ExpressionAnalyzer::optimizeGroupBy()
|
||||
}
|
||||
|
||||
if (group_exprs.empty())
|
||||
select_query->group_expression_list = nullptr;
|
||||
{
|
||||
/** Нельзя полностью убирать GROUP BY. Потому что если при этом даже агрегатных функций не было, то получится, что не будет агрегации.
|
||||
* Вместо этого оставим GROUP BY const.
|
||||
* Далее см. удаление констант в методе analyzeAggregation.
|
||||
*/
|
||||
|
||||
/// Нужно вставить константу, которая не является именем столбца таблицы. Такой случай редкий, но бывает.
|
||||
UInt64 unused_column = 0;
|
||||
String unused_column_name = toString(unused_column);
|
||||
|
||||
while (columns.end() != std::find_if(columns.begin(), columns.end(),
|
||||
[&unused_column_name](const NameAndTypePair & name_type) { return name_type.name == unused_column_name; }))
|
||||
{
|
||||
++unused_column;
|
||||
unused_column_name = toString(unused_column);
|
||||
}
|
||||
|
||||
select_query->group_expression_list = new ASTExpressionList;
|
||||
select_query->group_expression_list->children.push_back(new ASTLiteral(StringRange(), UInt64(unused_column)));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -1377,12 +1407,15 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
|
||||
{
|
||||
const String nested_table_name = ast->getColumnName();
|
||||
const String nested_table_alias = ast->getAliasOrColumnName();
|
||||
|
||||
if (nested_table_alias == nested_table_name && !typeid_cast<const ASTIdentifier *>(&*ast))
|
||||
throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name, ErrorCodes::ALIAS_REQUIRED);
|
||||
|
||||
if (array_join_alias_to_name.count(nested_table_alias) || aliases.count(nested_table_alias))
|
||||
throw Exception("Duplicate alias " + nested_table_alias, ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
|
||||
|
||||
array_join_alias_to_name[nested_table_alias] = nested_table_name;
|
||||
array_join_name_to_alias[nested_table_name] = nested_table_alias;
|
||||
}
|
||||
|
||||
ASTs & query_asts = select_query->children;
|
||||
@ -1425,6 +1458,7 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
|
||||
}
|
||||
|
||||
|
||||
/// Заполняет array_join_result_to_source: по каким столбцам-массивам размножить, и как их после этого назвать.
|
||||
void ExpressionAnalyzer::getArrayJoinedColumnsImpl(ASTPtr ast)
|
||||
{
|
||||
if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(&*ast))
|
||||
@ -1432,14 +1466,29 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(ASTPtr ast)
|
||||
if (node->kind == ASTIdentifier::Column)
|
||||
{
|
||||
String table_name = DataTypeNested::extractNestedTableName(node->name);
|
||||
|
||||
if (array_join_alias_to_name.count(node->name))
|
||||
array_join_result_to_source[node->name] = array_join_alias_to_name[node->name];
|
||||
{
|
||||
/// Был написан ARRAY JOIN со столбцом-массивом. Пример: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1
|
||||
array_join_result_to_source[node->name] = array_join_alias_to_name[node->name]; /// K1 -> ParsedParams.Key1
|
||||
}
|
||||
else if (array_join_alias_to_name.count(table_name))
|
||||
{
|
||||
String nested_column = DataTypeNested::extractNestedColumnName(node->name);
|
||||
array_join_result_to_source[node->name]
|
||||
/// Был написан ARRAY JOIN с вложенной таблицей. Пример: SELECT PP.Key1 FROM ... ARRAY JOIN ParsedParams AS PP
|
||||
String nested_column = DataTypeNested::extractNestedColumnName(node->name); /// Key1
|
||||
array_join_result_to_source[node->name] /// PP.Key1 -> ParsedParams.Key1
|
||||
= DataTypeNested::concatenateNestedName(array_join_alias_to_name[table_name], nested_column);
|
||||
}
|
||||
else if (array_join_name_to_alias.count(table_name))
|
||||
{
|
||||
/** Пример: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP.
|
||||
* То есть, в запросе используется исходный массив, размноженный по самому себе.
|
||||
*/
|
||||
|
||||
String nested_column = DataTypeNested::extractNestedColumnName(node->name); /// Key1
|
||||
array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
|
||||
DataTypeNested::concatenateNestedName(array_join_name_to_alias[table_name], nested_column)] = node->name;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -1481,10 +1530,12 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
|
||||
if (node->kind == ASTFunction::LAMBDA_EXPRESSION)
|
||||
throw Exception("Unexpected expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||
|
||||
/// Функция arrayJoin.
|
||||
if (node->kind == ASTFunction::ARRAY_JOIN)
|
||||
{
|
||||
if (node->arguments->children.size() != 1)
|
||||
throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
ASTPtr arg = node->arguments->children.at(0);
|
||||
getActionsImpl(arg, no_subqueries, only_consts, actions_stack);
|
||||
if (!only_consts)
|
||||
@ -1780,13 +1831,17 @@ void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAn
|
||||
}
|
||||
}
|
||||
|
||||
/// "Большой" ARRAY JOIN.
|
||||
void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const
|
||||
{
|
||||
NameSet result_columns;
|
||||
for (const auto & result_source : array_join_result_to_source)
|
||||
{
|
||||
/// Дать столбцам новые имена, если надо.
|
||||
if (result_source.first != result_source.second)
|
||||
actions->add(ExpressionAction::copyColumn(result_source.second, result_source.first));
|
||||
|
||||
/// Сделать ARRAY JOIN (заменить массивы на их внутренности) для столбцов в этими новыми именами.
|
||||
result_columns.insert(result_source.first);
|
||||
}
|
||||
|
||||
|
@ -1,16 +1,132 @@
|
||||
#include <DB/Interpreters/InterpreterCheckQuery.h>
|
||||
#include <DB/Parsers/ASTCheckQuery.h>
|
||||
#include <DB/Storages/StorageDistributed.h>
|
||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <DB/DataStreams/UnionBlockInputStream.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
|
||||
#include <openssl/sha.h>
|
||||
#include <deque>
|
||||
#include <array>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
InterpreterCheckQuery::InterpreterCheckQuery(DB::ASTPtr query_ptr_, DB::Context& context_) : query_ptr(query_ptr_), context(context_)
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Вспомогательная структура для оформления ответа на запрос DESCRIBE TABLE с Distributed-таблицей.
|
||||
/// Содержит информацию про локальную таблицу, которая была получена с одной реплики.
|
||||
struct TableDescription
|
||||
{
|
||||
TableDescription(const Block & block, const BlockExtraInfo & extra_info_)
|
||||
: extra_info(extra_info_)
|
||||
{
|
||||
const auto & name_column = typeid_cast<const ColumnString &>(*block.getByName("name").column);
|
||||
const auto & type_column = typeid_cast<const ColumnString &>(*block.getByName("type").column);
|
||||
const auto & default_type_column = typeid_cast<const ColumnString &>(*block.getByName("default_type").column);
|
||||
const auto & default_expression_column = typeid_cast<const ColumnString &>(*block.getByName("default_expression").column);
|
||||
|
||||
size_t row_count = block.rows();
|
||||
|
||||
names_with_types.reserve(name_column.byteSize() + type_column.byteSize() + (3 * row_count));
|
||||
|
||||
SHA512_CTX ctx;
|
||||
SHA512_Init(&ctx);
|
||||
|
||||
bool is_first = true;
|
||||
for (size_t i = 0; i < row_count; ++i)
|
||||
{
|
||||
const auto & name = name_column.getDataAt(i).toString();
|
||||
const auto & type = type_column.getDataAt(i).toString();
|
||||
const auto & default_type = default_type_column.getDataAt(i).toString();
|
||||
const auto & default_expression = default_expression_column.getDataAt(i).toString();
|
||||
|
||||
names_with_types.append(is_first ? "" : ", ");
|
||||
names_with_types.append(name);
|
||||
names_with_types.append(" ");
|
||||
names_with_types.append(type);
|
||||
|
||||
SHA512_Update(&ctx, reinterpret_cast<const unsigned char *>(name.data()), name.size());
|
||||
SHA512_Update(&ctx, reinterpret_cast<const unsigned char *>(type.data()), type.size());
|
||||
SHA512_Update(&ctx, reinterpret_cast<const unsigned char *>(default_type.data()), default_type.size());
|
||||
SHA512_Update(&ctx, reinterpret_cast<const unsigned char *>(default_expression.data()), default_expression.size());
|
||||
|
||||
is_first = false;
|
||||
}
|
||||
|
||||
SHA512_Final(hash.data(), &ctx);
|
||||
}
|
||||
|
||||
using Hash = std::array<unsigned char, SHA512_DIGEST_LENGTH>;
|
||||
|
||||
BlockExtraInfo extra_info;
|
||||
std::string names_with_types;
|
||||
Hash hash;
|
||||
UInt32 structure_class;
|
||||
};
|
||||
|
||||
inline bool operator<(const TableDescription & lhs, const TableDescription & rhs)
|
||||
{
|
||||
return lhs.hash < rhs.hash;
|
||||
}
|
||||
|
||||
using TableDescriptions = std::deque<TableDescription>;
|
||||
|
||||
}
|
||||
|
||||
InterpreterCheckQuery::InterpreterCheckQuery(DB::ASTPtr query_ptr_, DB::Context& context_)
|
||||
: query_ptr(query_ptr_), context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
Block InterpreterCheckQuery::getSampleBlock() const
|
||||
{
|
||||
Block block;
|
||||
ColumnWithTypeAndName col;
|
||||
|
||||
col.name = "status";
|
||||
col.type = new DataTypeUInt8;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "host_name";
|
||||
col.type = new DataTypeString;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "host_address";
|
||||
col.type = new DataTypeString;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "port";
|
||||
col.type = new DataTypeUInt16;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "user";
|
||||
col.type = new DataTypeString;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "structure_class";
|
||||
col.type = new DataTypeUInt32;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
col.name = "structure";
|
||||
col.type = new DataTypeString;
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterCheckQuery::execute()
|
||||
{
|
||||
ASTCheckQuery & alter = typeid_cast<ASTCheckQuery &>(*query_ptr);
|
||||
@ -19,14 +135,120 @@ BlockIO InterpreterCheckQuery::execute()
|
||||
|
||||
StoragePtr table = context.getTable(database_name, table_name);
|
||||
|
||||
result = Block{{ new ColumnUInt8, new DataTypeUInt8, "result" }};
|
||||
result.getByPosition(0).column->insert(Field(UInt64(table->checkData())));
|
||||
auto distributed_table = typeid_cast<StorageDistributed *>(&*table);
|
||||
if (distributed_table != nullptr)
|
||||
{
|
||||
/// Для таблиц с движком Distributed запрос CHECK TABLE отправляет запрос DESCRIBE TABLE на все реплики.
|
||||
/// Проверяется идентичность структур (имена столбцов + типы столбцов + типы по-умолчанию + выражения
|
||||
/// по-умолчанию) таблиц, на котороые смотрит распределённая таблица.
|
||||
|
||||
BlockIO res;
|
||||
res.in = new OneBlockInputStream(result);
|
||||
res.in_sample = result.cloneEmpty();
|
||||
const auto settings = context.getSettings();
|
||||
|
||||
return res;
|
||||
BlockInputStreams streams = distributed_table->describe(context, settings);
|
||||
streams[0] = new UnionBlockInputStream<StreamUnionMode::ExtraInfo>(streams, nullptr, settings.max_threads);
|
||||
streams.resize(1);
|
||||
|
||||
auto stream_ptr = dynamic_cast<IProfilingBlockInputStream *>(&*streams[0]);
|
||||
if (stream_ptr == nullptr)
|
||||
throw Exception("InterpreterCheckQuery: Internal error", ErrorCodes::LOGICAL_ERROR);
|
||||
auto & stream = *stream_ptr;
|
||||
|
||||
/// Получить все данные от запросов DESCRIBE TABLE.
|
||||
|
||||
TableDescriptions table_descriptions;
|
||||
|
||||
while (true)
|
||||
{
|
||||
if (stream.isCancelled())
|
||||
{
|
||||
BlockIO res;
|
||||
res.in = new OneBlockInputStream(result);
|
||||
return res;
|
||||
}
|
||||
|
||||
Block block = stream.read();
|
||||
if (!block)
|
||||
break;
|
||||
|
||||
BlockExtraInfo info = stream.getBlockExtraInfo();
|
||||
if (!info.is_valid)
|
||||
throw Exception("Received invalid block extra info", ErrorCodes::INVALID_BLOCK_EXTRA_INFO);
|
||||
|
||||
table_descriptions.emplace_back(block, info);
|
||||
}
|
||||
|
||||
if (table_descriptions.empty())
|
||||
throw Exception("Received empty data", ErrorCodes::RECEIVED_EMPTY_DATA);
|
||||
|
||||
/// Определить класс эквивалентности каждой структуры таблицы.
|
||||
|
||||
std::sort(table_descriptions.begin(), table_descriptions.end());
|
||||
|
||||
UInt32 structure_class = 0;
|
||||
|
||||
auto it = table_descriptions.begin();
|
||||
it->structure_class = structure_class;
|
||||
|
||||
auto prev = it;
|
||||
for (++it; it != table_descriptions.end(); ++it)
|
||||
{
|
||||
if (*prev < *it)
|
||||
++structure_class;
|
||||
it->structure_class = structure_class;
|
||||
prev = it;
|
||||
}
|
||||
|
||||
/// Составить результат.
|
||||
|
||||
ColumnPtr status_column = new ColumnUInt8;
|
||||
ColumnPtr host_name_column = new ColumnString;
|
||||
ColumnPtr host_address_column = new ColumnString;
|
||||
ColumnPtr port_column = new ColumnUInt16;
|
||||
ColumnPtr user_column = new ColumnString;
|
||||
ColumnPtr structure_class_column = new ColumnUInt32;
|
||||
ColumnPtr structure_column = new ColumnString;
|
||||
|
||||
/// Это значение равно 1, если структура нигде не отлчиается, а 0 в противном случае.
|
||||
UInt8 status_value = (structure_class == 0) ? 1 : 0;
|
||||
|
||||
for (const auto & desc : table_descriptions)
|
||||
{
|
||||
status_column->insert(static_cast<UInt64>(status_value));
|
||||
structure_class_column->insert(static_cast<UInt64>(desc.structure_class));
|
||||
host_name_column->insert(desc.extra_info.host);
|
||||
host_address_column->insert(desc.extra_info.resolved_address);
|
||||
port_column->insert(static_cast<UInt64>(desc.extra_info.port));
|
||||
user_column->insert(desc.extra_info.user);
|
||||
structure_column->insert(desc.names_with_types);
|
||||
}
|
||||
|
||||
Block block;
|
||||
|
||||
block.insert(ColumnWithTypeAndName(status_column, new DataTypeUInt8, "status"));
|
||||
block.insert(ColumnWithTypeAndName(host_name_column, new DataTypeString, "host_name"));
|
||||
block.insert(ColumnWithTypeAndName(host_address_column, new DataTypeString, "host_address"));
|
||||
block.insert(ColumnWithTypeAndName(port_column, new DataTypeUInt16, "port"));
|
||||
block.insert(ColumnWithTypeAndName(user_column, new DataTypeString, "user"));
|
||||
block.insert(ColumnWithTypeAndName(structure_class_column, new DataTypeUInt32, "structure_class"));
|
||||
block.insert(ColumnWithTypeAndName(structure_column, new DataTypeString, "structure"));
|
||||
|
||||
BlockIO res;
|
||||
res.in = new OneBlockInputStream(block);
|
||||
res.in_sample = getSampleBlock();
|
||||
|
||||
return res;
|
||||
}
|
||||
else
|
||||
{
|
||||
result = Block{{ new ColumnUInt8, new DataTypeUInt8, "result" }};
|
||||
result.getByPosition(0).column->insert(Field(UInt64(table->checkData())));
|
||||
|
||||
BlockIO res;
|
||||
res.in = new OneBlockInputStream(result);
|
||||
res.in_sample = result.cloneEmpty();
|
||||
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -33,6 +33,7 @@
|
||||
#include <DB/DataTypes/DataTypeNested.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/DataTypes/DataTypeFactory.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -331,18 +332,21 @@ InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(
|
||||
/// specific code for different data types, e.g. toFixedString(col, N) for DataTypeFixedString
|
||||
if (const auto fixed_string = typeid_cast<const DataTypeFixedString *>(data_type_ptr))
|
||||
{
|
||||
const auto conversion_function_name = "toFixedString";
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction(
|
||||
conversion_function_name,
|
||||
"toFixedString",
|
||||
ASTPtr{new ASTIdentifier{{}, tmp_column_name}},
|
||||
ASTPtr{new ASTLiteral{{}, fixed_string->getN()}}),
|
||||
final_column_name));
|
||||
}
|
||||
else if (typeid_cast<const DataTypeArray *>(data_type_ptr))
|
||||
{
|
||||
/// do not perform conversion on arrays, require exact type match
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
col_decl.default_expression->clone(), final_column_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
/// @todo fix for parametric types, results in broken code, i.e. toArray(ElementType)(col)
|
||||
const auto conversion_function_name = "to" + data_type_ptr->getName();
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
@ -370,15 +374,29 @@ InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(
|
||||
const auto name_and_type_ptr = column.first;
|
||||
const auto col_decl_ptr = column.second;
|
||||
|
||||
if (col_decl_ptr->type)
|
||||
{
|
||||
const auto & tmp_column = block.getByName(col_decl_ptr->name + "_tmp");
|
||||
const auto & column_name = col_decl_ptr->name;
|
||||
const auto has_explicit_type = nullptr != col_decl_ptr->type;
|
||||
auto & explicit_type = name_and_type_ptr->type;
|
||||
|
||||
/// type mismatch between explicitly specified and deduced type, add conversion
|
||||
if (name_and_type_ptr->type->getName() != tmp_column.type->getName())
|
||||
/// if column declaration contains explicit type, name_and_type_ptr->type is not null
|
||||
if (has_explicit_type)
|
||||
{
|
||||
const auto & tmp_column = block.getByName(column_name + "_tmp");
|
||||
const auto & deduced_type = tmp_column.type;
|
||||
|
||||
/// type mismatch between explicitly specified and deduced type, add conversion for non-array types
|
||||
if (explicit_type->getName() != deduced_type->getName())
|
||||
{
|
||||
col_decl_ptr->default_expression = makeASTFunction(
|
||||
"to" + name_and_type_ptr->type->getName(),
|
||||
/// foolproof against defaulting array columns incorrectly
|
||||
if (typeid_cast<const DataTypeArray *>(explicit_type.get()))
|
||||
throw Exception{
|
||||
"Default expression type mismatch for column " + column_name +
|
||||
". Expected " + explicit_type->getName() + ", deduced " +
|
||||
deduced_type->getName(),
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
|
||||
col_decl_ptr->default_expression = makeASTFunction("to" + explicit_type->getName(),
|
||||
col_decl_ptr->default_expression);
|
||||
|
||||
col_decl_ptr->children.clear();
|
||||
@ -387,9 +405,10 @@ InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(
|
||||
}
|
||||
}
|
||||
else
|
||||
name_and_type_ptr->type = block.getByName(name_and_type_ptr->name).type;
|
||||
/// no explicit type, name_and_type_ptr->type is null, set to deduced type
|
||||
explicit_type = block.getByName(column_name).type;
|
||||
|
||||
defaults.emplace(col_decl_ptr->name, ColumnDefault{
|
||||
defaults.emplace(column_name, ColumnDefault{
|
||||
columnDefaultTypeFromString(col_decl_ptr->default_specifier),
|
||||
col_decl_ptr->default_expression
|
||||
});
|
||||
|
@ -471,6 +471,17 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
/// Если есть агрегация, выполняем выражения в SELECT и ORDER BY на инициировавшем сервере, иначе - на серверах-источниках.
|
||||
query_analyzer->appendSelect(chain, need_aggregate ? !second_stage : !first_stage);
|
||||
selected_columns = chain.getLastStep().required_output;
|
||||
|
||||
/** Если есть LIMIT, то ARRAY JOIN нельзя переносить позже ORDER BY
|
||||
* (так как он меняет количество строк и частичная сортировка ORDER с LIMIT-ом перестают правильно работать).
|
||||
*/
|
||||
if (query.order_expression_list && query.limit_length && query.array_join_expression_list)
|
||||
{
|
||||
/// Завершаем цепочку действий, в рамках которой ARRAY JOIN может переноситься.
|
||||
chain.finalize();
|
||||
chain.clear();
|
||||
}
|
||||
|
||||
has_order_by = query_analyzer->appendOrderBy(chain, need_aggregate ? !second_stage : !first_stage);
|
||||
before_order_and_select = chain.getLastActions();
|
||||
chain.addStep();
|
||||
@ -1047,7 +1058,7 @@ void InterpreterSelectQuery::executeUnion()
|
||||
/// Если до сих пор есть несколько потоков, то объединяем их в один
|
||||
if (hasMoreThanOneStream())
|
||||
{
|
||||
streams[0] = new UnionBlockInputStream(streams, stream_with_non_joined_data, settings.max_threads);
|
||||
streams[0] = new UnionBlockInputStream<>(streams, stream_with_non_joined_data, settings.max_threads);
|
||||
stream_with_non_joined_data = nullptr;
|
||||
streams.resize(1);
|
||||
union_within_single_query = false;
|
||||
|
@ -245,7 +245,6 @@ void LogicalExpressionsOptimizer::cleanupOrExpressions()
|
||||
garbage_map.reserve(processed_count);
|
||||
for (const auto & chain : disjunctive_equality_chains_map)
|
||||
{
|
||||
const auto & equalities = chain.second;
|
||||
if (!chain.second.is_processed)
|
||||
continue;
|
||||
|
||||
|
16
dbms/src/ODBC/README
Normal file
16
dbms/src/ODBC/README
Normal file
@ -0,0 +1,16 @@
|
||||
Install unixodbc.
|
||||
|
||||
g++ -std=gnu++1y -Wall -g -shared -fPIC -lPocoFoundation -lPocoNet -o odbc.so odbc.cpp
|
||||
|
||||
~/.odbc.ini:
|
||||
|
||||
[ClickHouse]
|
||||
Driver = /home/milovidov/work/metrika-core/metrica/src/dbms/src/ODBC/odbc.so
|
||||
Description = ClickHouse driver
|
||||
DATABASE = default
|
||||
HOST = localhost
|
||||
PORT = 9000
|
||||
FRAMED = 0
|
||||
|
||||
Run
|
||||
iusql -v ClickHouse
|
1750
dbms/src/ODBC/odbc.cpp
Normal file
1750
dbms/src/ODBC/odbc.cpp
Normal file
File diff suppressed because it is too large
Load Diff
@ -8,7 +8,6 @@ using namespace DB;
|
||||
|
||||
bool ParserCheckQuery::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
||||
{
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
ParserString s_check("CHECK", true, true);
|
||||
ParserString s_table("TABLE", true, true);
|
||||
ParserString s_dot(".");
|
||||
@ -46,6 +45,12 @@ bool ParserCheckQuery::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr &
|
||||
query->table = typeid_cast<ASTIdentifier &>(*table).name;
|
||||
}
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
/// FORMAT format_name
|
||||
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
node = query;
|
||||
return true;
|
||||
}
|
||||
|
28
dbms/src/Parsers/ParserQueryWithOutput.cpp
Normal file
28
dbms/src/Parsers/ParserQueryWithOutput.cpp
Normal file
@ -0,0 +1,28 @@
|
||||
#include <DB/Parsers/ParserQueryWithOutput.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Parsers/ExpressionElementParsers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserQueryWithOutput::parseFormat(ASTQueryWithOutput & query, Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
||||
{
|
||||
ParserString s_format("FORMAT", true, true);
|
||||
|
||||
if (s_format.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
ParserIdentifier format_p;
|
||||
|
||||
if (!format_p.parse(pos, end, query.format, max_parsed_pos, expected))
|
||||
return false;
|
||||
typeid_cast<ASTIdentifier &>(*(query.format)).kind = ASTIdentifier::Format;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
@ -19,7 +19,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ASTSelectQuery * select_query = new ASTSelectQuery;
|
||||
node = select_query;
|
||||
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
ParserString s_select("SELECT", true, true);
|
||||
ParserString s_distinct("DISTINCT", true, true);
|
||||
ParserString s_from("FROM", true, true);
|
||||
@ -39,7 +38,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ParserString s_order("ORDER", true, true);
|
||||
ParserString s_limit("LIMIT", true, true);
|
||||
ParserString s_settings("SETTINGS", true, true);
|
||||
ParserString s_format("FORMAT", true, true);
|
||||
ParserString s_union("UNION", true, true);
|
||||
ParserString s_all("ALL", true, true);
|
||||
|
||||
@ -310,22 +308,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
bool has_format = false;
|
||||
|
||||
/// FORMAT format_name
|
||||
if (s_format.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
ParserIdentifier format_p;
|
||||
|
||||
if (!format_p.parse(pos, end, select_query->format, max_parsed_pos, expected))
|
||||
return false;
|
||||
typeid_cast<ASTIdentifier &>(*select_query->format).kind = ASTIdentifier::Format;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
has_format = true;
|
||||
}
|
||||
if (!parseFormat(*select_query, pos, end, node, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
// UNION ALL select query
|
||||
if (s_union.ignore(pos, end, max_parsed_pos, expected))
|
||||
@ -334,7 +319,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
|
||||
if (s_all.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
if (has_format)
|
||||
if (!select_query->format.isNull())
|
||||
{
|
||||
/// FORMAT может быть задан только в последнем запросе цепочки UNION ALL.
|
||||
expected = "FORMAT only in the last SELECT of the UNION ALL chain";
|
||||
|
@ -15,20 +15,17 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
|
||||
{
|
||||
Pos begin = pos;
|
||||
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
ParserString s_show("SHOW", true, true);
|
||||
ParserString s_tables("TABLES", true, true);
|
||||
ParserString s_databases("DATABASES", true, true);
|
||||
ParserString s_from("FROM", true, true);
|
||||
ParserString s_not("NOT", true, true);
|
||||
ParserString s_like("LIKE", true, true);
|
||||
ParserString s_format("FORMAT", true, true);
|
||||
ParserStringLiteral like_p;
|
||||
ParserIdentifier name_p;
|
||||
|
||||
ASTPtr like;
|
||||
ASTPtr database;
|
||||
ASTPtr format;
|
||||
|
||||
ASTShowTablesQuery * query = new ASTShowTablesQuery;
|
||||
ASTPtr query_ptr = query;
|
||||
@ -80,18 +77,9 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (s_format.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
ParserIdentifier format_p;
|
||||
|
||||
if (!format_p.parse(pos, end, format, max_parsed_pos, expected))
|
||||
return false;
|
||||
typeid_cast<ASTIdentifier &>(*format).kind = ASTIdentifier::Format;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
/// FORMAT format_name
|
||||
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
query->range = StringRange(begin, pos);
|
||||
|
||||
@ -99,11 +87,8 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
|
||||
query->from = typeid_cast<ASTIdentifier &>(*database).name;
|
||||
if (like)
|
||||
query->like = safeGet<const String &>(typeid_cast<ASTLiteral &>(*like).value);
|
||||
if (format)
|
||||
{
|
||||
query->format = format;
|
||||
query->children.push_back(format);
|
||||
}
|
||||
if (query->format)
|
||||
query->children.push_back(query->format);
|
||||
|
||||
node = query_ptr;
|
||||
|
||||
|
@ -13,20 +13,17 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
|
||||
{
|
||||
Pos begin = pos;
|
||||
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
ParserString s_exists("EXISTS", true, true);
|
||||
ParserString s_describe("DESCRIBE", true, true);
|
||||
ParserString s_desc("DESC", true, true);
|
||||
ParserString s_show("SHOW", true, true);
|
||||
ParserString s_create("CREATE", true, true);
|
||||
ParserString s_table("TABLE", true, true);
|
||||
ParserString s_format("FORMAT", true, true);
|
||||
ParserString s_dot(".");
|
||||
ParserIdentifier name_p;
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr table;
|
||||
ASTPtr format;
|
||||
ASTPtr query_ptr;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
@ -53,6 +50,7 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
|
||||
return false;
|
||||
}
|
||||
|
||||
ASTQueryWithTableAndOutput * query = dynamic_cast<ASTQueryWithTableAndOutput *>(&*query_ptr);
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
@ -76,20 +74,9 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (s_format.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
ParserIdentifier format_p;
|
||||
|
||||
if (!format_p.parse(pos, end, format, max_parsed_pos, expected))
|
||||
return false;
|
||||
typeid_cast<ASTIdentifier &>(*format).kind = ASTIdentifier::Format;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
ASTQueryWithTableAndOutput * query = dynamic_cast<ASTQueryWithTableAndOutput *>(&*query_ptr);
|
||||
/// FORMAT format_name
|
||||
if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
query->range = StringRange(begin, pos);
|
||||
|
||||
@ -97,11 +84,8 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
|
||||
query->database = typeid_cast<ASTIdentifier &>(*database).name;
|
||||
if (table)
|
||||
query->table = typeid_cast<ASTIdentifier &>(*table).name;
|
||||
if (format)
|
||||
{
|
||||
query->format = format;
|
||||
query->children.push_back(format);
|
||||
}
|
||||
if (query->format)
|
||||
query->children.push_back(query->format);
|
||||
|
||||
node = query_ptr;
|
||||
|
||||
|
@ -8,12 +8,12 @@
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
void AlterCommand::apply(NamesAndTypesList & columns,
|
||||
NamesAndTypesList & materialized_columns,
|
||||
NamesAndTypesList & alias_columns,
|
||||
ColumnDefaults & column_defaults) const
|
||||
void AlterCommand::apply(
|
||||
NamesAndTypesList & columns, NamesAndTypesList & materialized_columns, NamesAndTypesList & alias_columns,
|
||||
ColumnDefaults & column_defaults) const
|
||||
{
|
||||
if (type == ADD)
|
||||
{
|
||||
@ -27,7 +27,7 @@ namespace DB
|
||||
exists_in(alias_columns))
|
||||
{
|
||||
throw Exception{
|
||||
"Cannot add column " + column_name + ": column with this name already exisits.",
|
||||
"Cannot add column " + column_name + ": column with this name already exists",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
@ -99,40 +99,37 @@ namespace DB
|
||||
}
|
||||
else if (type == MODIFY)
|
||||
{
|
||||
const auto it = column_defaults.find(column_name);
|
||||
const auto had_default_expr = it != column_defaults.end();
|
||||
const auto old_default_type = had_default_expr ? it->second.type : ColumnDefaultType{};
|
||||
const auto default_it = column_defaults.find(column_name);
|
||||
const auto had_default_expr = default_it != std::end(column_defaults);
|
||||
const auto old_default_type = had_default_expr ? default_it->second.type : ColumnDefaultType{};
|
||||
|
||||
/// allow conversion between DEFAULT and MATERIALIZED
|
||||
const auto default_materialized_conversion =
|
||||
(old_default_type == ColumnDefaultType::Default && default_type == ColumnDefaultType::Materialized) ||
|
||||
(old_default_type == ColumnDefaultType::Materialized && default_type == ColumnDefaultType::Default);
|
||||
|
||||
if (old_default_type != default_type && !default_materialized_conversion)
|
||||
throw Exception{"Cannot change column default specifier from " + toString(old_default_type) +
|
||||
" to " + toString(default_type), ErrorCodes::INCORRECT_QUERY};
|
||||
/// target column list
|
||||
auto & new_columns = default_type == ColumnDefaultType::Default ?
|
||||
columns : default_type == ColumnDefaultType::Materialized ?
|
||||
materialized_columns : alias_columns;
|
||||
|
||||
/// find column or throw exception
|
||||
const auto find_column = [this] (NamesAndTypesList & columns) {
|
||||
const auto it = std::find_if(columns.begin(), columns.end(),
|
||||
std::bind(namesEqual, std::cref(column_name), std::placeholders::_1) );
|
||||
if (it == columns.end())
|
||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to modify.",
|
||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to modify",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return it;
|
||||
};
|
||||
|
||||
/// remove from the old list, add to the new list in case of DEFAULT <-> MATERIALIZED alteration
|
||||
if (default_materialized_conversion)
|
||||
/// if default types differ, remove column from the old list, then add to the new list
|
||||
if (default_type != old_default_type)
|
||||
{
|
||||
const auto was_default = old_default_type == ColumnDefaultType::Default;
|
||||
auto & old_columns = was_default ? columns : materialized_columns;
|
||||
auto & new_columns = was_default ? materialized_columns : columns;
|
||||
/// source column list
|
||||
auto & old_columns = old_default_type == ColumnDefaultType::Default ?
|
||||
columns : old_default_type == ColumnDefaultType::Materialized ?
|
||||
materialized_columns : alias_columns;
|
||||
|
||||
const auto column_it = find_column(old_columns);
|
||||
new_columns.emplace_back(*column_it);
|
||||
old_columns.erase(column_it);
|
||||
const auto old_column_it = find_column(old_columns);
|
||||
new_columns.emplace_back(*old_column_it);
|
||||
old_columns.erase(old_column_it);
|
||||
|
||||
/// do not forget to change the default type of old column
|
||||
if (had_default_expr)
|
||||
@ -140,19 +137,17 @@ namespace DB
|
||||
}
|
||||
|
||||
/// find column in one of three column lists
|
||||
const auto column_it = find_column(
|
||||
default_type == ColumnDefaultType::Default ? columns :
|
||||
default_type == ColumnDefaultType::Materialized ? materialized_columns :
|
||||
alias_columns);
|
||||
|
||||
const auto column_it = find_column(new_columns);
|
||||
column_it->type = data_type;
|
||||
|
||||
/// remove, add or update default_expression
|
||||
if (!default_expression && had_default_expr)
|
||||
/// new column has no default expression, remove it from column_defaults along with it's type
|
||||
column_defaults.erase(column_name);
|
||||
else if (default_expression && !had_default_expr)
|
||||
/// new column has a default expression while the old one had not, add it it column_defaults
|
||||
column_defaults.emplace(column_name, ColumnDefault{default_type, default_expression});
|
||||
else if (had_default_expr)
|
||||
/// both old and new columns have default expression, update it
|
||||
column_defaults[column_name].expression = default_expression;
|
||||
}
|
||||
else
|
||||
@ -184,7 +179,7 @@ namespace DB
|
||||
columns.insert(std::end(columns), std::begin(table->alias_columns), std::end(table->alias_columns));
|
||||
auto defaults = table->column_defaults;
|
||||
|
||||
std::vector<std::pair<String, AlterCommand *>> defaulted_columns{};
|
||||
std::vector<std::pair<NameAndTypePair, AlterCommand *>> defaulted_columns{};
|
||||
|
||||
ASTPtr default_expr_list{new ASTExpressionList};
|
||||
default_expr_list->children.reserve(defaults.size());
|
||||
@ -193,32 +188,44 @@ namespace DB
|
||||
{
|
||||
if (command.type == AlterCommand::ADD || command.type == AlterCommand::MODIFY)
|
||||
{
|
||||
if (command.type == AlterCommand::MODIFY)
|
||||
const auto & column_name = command.column_name;
|
||||
const auto column_it = std::find_if(std::begin(columns), std::end(columns),
|
||||
std::bind(AlterCommand::namesEqual, std::cref(command.column_name), std::placeholders::_1));
|
||||
|
||||
if (command.type == AlterCommand::ADD)
|
||||
{
|
||||
if (std::end(columns) != column_it)
|
||||
throw Exception{
|
||||
"Cannot add column " + column_name + ": column with this name already exisits",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
else if (command.type == AlterCommand::MODIFY)
|
||||
{
|
||||
const auto it = std::find_if(std::begin(columns), std::end(columns),
|
||||
std::bind(AlterCommand::namesEqual, std::cref(command.column_name), std::placeholders::_1));
|
||||
|
||||
if (it == std::end(columns))
|
||||
throw Exception("Wrong column name. Cannot find column " + command.column_name + " to modify.",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (std::end(columns) == column_it)
|
||||
throw Exception{
|
||||
"Wrong column name. Cannot find column " + column_name + " to modify",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
|
||||
columns.erase(it);
|
||||
defaults.erase(command.column_name);
|
||||
columns.erase(column_it);
|
||||
defaults.erase(column_name);
|
||||
}
|
||||
|
||||
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
||||
columns.emplace_back(command.column_name, command.data_type ? command.data_type : new DataTypeUInt8);
|
||||
columns.emplace_back(column_name, command.data_type ? command.data_type : new DataTypeUInt8);
|
||||
|
||||
if (command.default_expression)
|
||||
{
|
||||
if (command.data_type)
|
||||
{
|
||||
const auto & final_column_name = command.column_name;
|
||||
const auto & final_column_name = column_name;
|
||||
const auto tmp_column_name = final_column_name + "_tmp";
|
||||
const auto data_type_ptr = command.data_type.get();
|
||||
const auto column_type_raw_ptr = command.data_type.get();
|
||||
|
||||
/// specific code for different data types, e.g. toFixedString(col, N) for DataTypeFixedString
|
||||
if (const auto fixed_string = typeid_cast<const DataTypeFixedString *>(data_type_ptr))
|
||||
if (const auto fixed_string = typeid_cast<const DataTypeFixedString *>(column_type_raw_ptr))
|
||||
{
|
||||
const auto conversion_function_name = "toFixedString";
|
||||
|
||||
@ -229,10 +236,15 @@ namespace DB
|
||||
ASTPtr{new ASTLiteral{{}, fixed_string->getN()}}),
|
||||
final_column_name));
|
||||
}
|
||||
else if (typeid_cast<const DataTypeArray *>(column_type_raw_ptr))
|
||||
{
|
||||
/// do not perform conversion on arrays, require exact type match
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
command.default_expression->clone(), final_column_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
/// @todo fix for parametric types, results in broken codem, i.e. toArray(ElementType)(col)
|
||||
const auto conversion_function_name = "to" + data_type_ptr->getName();
|
||||
const auto conversion_function_name = "to" + column_type_raw_ptr->getName();
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction(conversion_function_name, ASTPtr{new ASTIdentifier{{}, tmp_column_name}}),
|
||||
@ -241,14 +253,15 @@ namespace DB
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
|
||||
|
||||
defaulted_columns.emplace_back(command.column_name, &command);
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, command.data_type}, &command);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// no type explicitly specified, will deduce later
|
||||
default_expr_list->children.emplace_back(
|
||||
setAlias(command.default_expression->clone(), command.column_name));
|
||||
setAlias(command.default_expression->clone(), column_name));
|
||||
|
||||
defaulted_columns.emplace_back(command.column_name, &command);
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, nullptr}, &command);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -271,7 +284,7 @@ namespace DB
|
||||
++it;
|
||||
|
||||
if (!found)
|
||||
throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop.",
|
||||
throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
@ -285,15 +298,38 @@ namespace DB
|
||||
return AlterCommand::namesEqual(column_name, name_type);
|
||||
});
|
||||
const auto tmp_column_name = column_name + "_tmp";
|
||||
const auto conversion_function_name = "to" + column_it->type->getName();
|
||||
const auto & column_type_ptr = column_it->type;
|
||||
const auto column_type_raw_ptr = column_type_ptr.get();
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction(conversion_function_name, ASTPtr{new ASTIdentifier{{}, tmp_column_name}}),
|
||||
column_name));
|
||||
/// specific code for different data types, e.g. toFixedString(col, N) for DataTypeFixedString
|
||||
if (const auto fixed_string = typeid_cast<const DataTypeFixedString *>(column_type_raw_ptr))
|
||||
{
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction("toFixedString",
|
||||
ASTPtr{new ASTIdentifier{{}, tmp_column_name}},
|
||||
ASTPtr{new ASTLiteral{{}, fixed_string->getN()}}),
|
||||
column_name));
|
||||
}
|
||||
else if (typeid_cast<const DataTypeArray *>(column_type_raw_ptr))
|
||||
{
|
||||
/// do not perform conversion on arrays, require exact type match
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
col_def.second.expression->clone(),
|
||||
column_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto conversion_function_name = "to" + column_it->type->getName();
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction(conversion_function_name,
|
||||
ASTPtr{new ASTIdentifier{{}, tmp_column_name}}),
|
||||
column_name));
|
||||
}
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(col_def.second.expression->clone(), tmp_column_name));
|
||||
|
||||
defaulted_columns.emplace_back(column_name, nullptr);
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, column_type_ptr}, nullptr);
|
||||
}
|
||||
|
||||
const auto actions = ExpressionAnalyzer{default_expr_list, context, {}, columns}.getActions(true);
|
||||
@ -302,41 +338,62 @@ namespace DB
|
||||
/// set deduced types, modify default expression if necessary
|
||||
for (auto & defaulted_column : defaulted_columns)
|
||||
{
|
||||
const auto & column_name = defaulted_column.first;
|
||||
const auto command_ptr = defaulted_column.second;
|
||||
const auto & column = block.getByName(column_name);
|
||||
const auto & name_and_type = defaulted_column.first;
|
||||
AlterCommand * & command_ptr = defaulted_column.second;
|
||||
|
||||
const auto & column_name = name_and_type.name;
|
||||
const auto has_explicit_type = nullptr != name_and_type.type;
|
||||
|
||||
/// default expression on old column
|
||||
if (!command_ptr)
|
||||
if (has_explicit_type)
|
||||
{
|
||||
const auto & explicit_type = name_and_type.type;
|
||||
const auto & tmp_column = block.getByName(column_name + "_tmp");
|
||||
const auto & deduced_type = tmp_column.type;
|
||||
|
||||
// column not specified explicitly in the ALTER query may require default_expression modification
|
||||
if (column.type->getName() != tmp_column.type->getName())
|
||||
if (explicit_type->getName() != deduced_type->getName())
|
||||
{
|
||||
const auto it = defaults.find(column_name);
|
||||
this->push_back(AlterCommand{
|
||||
AlterCommand::MODIFY, column_name, column.type, it->second.type,
|
||||
makeASTFunction("to" + column.type->getName(), it->second.expression),
|
||||
});
|
||||
}
|
||||
}
|
||||
else if (command_ptr && command_ptr->data_type)
|
||||
{
|
||||
const auto & tmp_column = block.getByName(column_name + "_tmp");
|
||||
const auto default_it = defaults.find(column_name);
|
||||
|
||||
/// type mismatch between explicitly specified and deduced type, add conversion
|
||||
if (column.type->getName() != tmp_column.type->getName())
|
||||
{
|
||||
command_ptr->default_expression = makeASTFunction(
|
||||
"to" + column.type->getName(),
|
||||
command_ptr->default_expression->clone());
|
||||
/// column has no associated alter command, let's create it
|
||||
if (!command_ptr)
|
||||
{
|
||||
/// add a new alter command to modify existing column
|
||||
this->emplace_back(AlterCommand{
|
||||
AlterCommand::MODIFY, column_name, explicit_type,
|
||||
default_it->second.type, default_it->second.expression
|
||||
});
|
||||
|
||||
command_ptr = &this->back();
|
||||
}
|
||||
|
||||
if (const auto fixed_string = typeid_cast<const DataTypeFixedString *>(explicit_type.get()))
|
||||
{
|
||||
command_ptr->default_expression = makeASTFunction("toFixedString",
|
||||
command_ptr->default_expression->clone(),
|
||||
ASTPtr{new ASTLiteral{{}, fixed_string->getN()}});
|
||||
}
|
||||
else if (typeid_cast<const DataTypeArray *>(explicit_type.get()))
|
||||
{
|
||||
/// foolproof against defaulting array columns incorrectly
|
||||
throw Exception{
|
||||
"Default expression type mismatch for column " + column_name + ". Expected " +
|
||||
explicit_type->getName() + ", deduced " + deduced_type->getName(),
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
}
|
||||
else
|
||||
{
|
||||
command_ptr->default_expression = makeASTFunction("to" + explicit_type->getName(),
|
||||
command_ptr->default_expression->clone());
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// just set deduced type
|
||||
command_ptr->data_type = column.type;
|
||||
command_ptr->data_type = block.getByName(column_name).type;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -5,6 +5,9 @@
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Parsers/ASTNameTypePair.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <ext/map.hpp>
|
||||
#include <ext/identity.hpp>
|
||||
#include <ext/collection_cast.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,25 +15,19 @@ namespace DB
|
||||
|
||||
NamesAndTypesList ITableDeclaration::getColumnsList() const
|
||||
{
|
||||
auto columns = getColumnsListImpl();
|
||||
columns.insert(std::end(columns), std::begin(materialized_columns), std::end(materialized_columns));
|
||||
|
||||
return columns;
|
||||
return ext::collection_cast<NamesAndTypesList>(getColumnsListRange());
|
||||
}
|
||||
|
||||
|
||||
ITableDeclaration::ColumnsListRange ITableDeclaration::getColumnsListIterator() const
|
||||
ITableDeclaration::ColumnsListRange ITableDeclaration::getColumnsListRange() const
|
||||
{
|
||||
const auto & columns = getColumnsListImpl();
|
||||
return boost::join(
|
||||
boost::iterator_range<NamesAndTypesList::const_iterator>(columns.begin(), columns.end()),
|
||||
boost::iterator_range<NamesAndTypesList::const_iterator>(std::begin(materialized_columns), std::end(materialized_columns)));
|
||||
return boost::join(getColumnsListImpl(), materialized_columns);
|
||||
}
|
||||
|
||||
|
||||
bool ITableDeclaration::hasRealColumn(const String & column_name) const
|
||||
{
|
||||
for (auto & it : getColumnsListIterator())
|
||||
for (auto & it : getColumnsListRange())
|
||||
if (it.name == column_name)
|
||||
return true;
|
||||
return false;
|
||||
@ -39,16 +36,13 @@ bool ITableDeclaration::hasRealColumn(const String & column_name) const
|
||||
|
||||
Names ITableDeclaration::getColumnNamesList() const
|
||||
{
|
||||
Names res;
|
||||
for (auto & it : getColumnsListIterator())
|
||||
res.push_back(it.name);
|
||||
return res;
|
||||
return ext::map<Names>(getColumnsListRange(), [] (const auto & it) { return it.name; });
|
||||
}
|
||||
|
||||
|
||||
NameAndTypePair ITableDeclaration::getRealColumn(const String & column_name) const
|
||||
{
|
||||
for (auto & it : getColumnsListIterator())
|
||||
for (auto & it : getColumnsListRange())
|
||||
if (it.name == column_name)
|
||||
return it;
|
||||
throw Exception("There is no column " + column_name + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
@ -85,7 +79,7 @@ NameAndTypePair ITableDeclaration::getColumn(const String & column_name) const
|
||||
|
||||
const DataTypePtr ITableDeclaration::getDataTypeByName(const String & column_name) const
|
||||
{
|
||||
for (const auto & column : getColumnsListIterator())
|
||||
for (const auto & column : getColumnsListRange())
|
||||
if (column.name == column_name)
|
||||
return column.type;
|
||||
|
||||
@ -97,7 +91,7 @@ Block ITableDeclaration::getSampleBlock() const
|
||||
{
|
||||
Block res;
|
||||
|
||||
for (const auto & col : getColumnsListIterator())
|
||||
for (const auto & col : getColumnsListRange())
|
||||
res.insert({ col.type->createColumn(), col.type, col.name });
|
||||
|
||||
return res;
|
||||
|
@ -557,6 +557,8 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
|
||||
if (expression)
|
||||
{
|
||||
MarkRanges ranges(1, MarkRange(0, part->size));
|
||||
/** @todo expression->getRequiedColumns may contain integer width columns for FixedString(N) type which after
|
||||
* passing them to ITableDeclaration::check will trigger and exception about unknown column `N` */
|
||||
BlockInputStreamPtr part_in = new MergeTreeBlockInputStream(full_path + part->name + '/',
|
||||
DEFAULT_MERGE_BLOCK_SIZE, expression->getRequiredColumns(), *this, part, ranges, false, nullptr, "", false, 0, DBMS_DEFAULT_BUFFER_SIZE);
|
||||
ExpressionBlockInputStream in(part_in, expression);
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
||||
#include <DB/DataStreams/MaterializingBlockInputStream.h>
|
||||
#include <DB/DataStreams/BlockExtraInfoInputStream.h>
|
||||
|
||||
#include <DB/Storages/StorageDistributed.h>
|
||||
#include <DB/Storages/VirtualColumnFactory.h>
|
||||
@ -8,9 +9,11 @@
|
||||
#include <DB/Common/escapeForFileName.h>
|
||||
#include <DB/Parsers/ASTInsertQuery.h>
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
|
||||
|
||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
||||
#include <DB/Interpreters/InterpreterDescribeQuery.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
|
||||
#include <DB/Core/Field.h>
|
||||
@ -49,6 +52,17 @@ namespace
|
||||
|
||||
return modified_query_ast;
|
||||
}
|
||||
|
||||
BlockExtraInfo toBlockExtraInfo(const Cluster::Address & address)
|
||||
{
|
||||
BlockExtraInfo block_extra_info;
|
||||
block_extra_info.host = address.host_name;
|
||||
block_extra_info.resolved_address = address.resolved_address.toString();
|
||||
block_extra_info.port = address.port;
|
||||
block_extra_info.user = address.user;
|
||||
block_extra_info.is_valid = true;
|
||||
return block_extra_info;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -66,7 +80,7 @@ StorageDistributed::StorageDistributed(
|
||||
context(context_), cluster(cluster_),
|
||||
sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, nullptr, *columns).getActions(false) : nullptr),
|
||||
sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}),
|
||||
write_enabled(!data_path_.empty() && (cluster.getLocalNodesNum() + cluster.pools.size() < 2 || sharding_key_)),
|
||||
write_enabled(!data_path_.empty() && (((cluster.getLocalShardCount() + cluster.getRemoteShardCount()) < 2) || sharding_key_)),
|
||||
path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(name) + '/'))
|
||||
{
|
||||
createDirectoryMonitors();
|
||||
@ -90,7 +104,7 @@ StorageDistributed::StorageDistributed(
|
||||
context(context_), cluster(cluster_),
|
||||
sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, nullptr, *columns).getActions(false) : nullptr),
|
||||
sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}),
|
||||
write_enabled(!data_path_.empty() && (cluster.getLocalNodesNum() + cluster.pools.size() < 2 || sharding_key_)),
|
||||
write_enabled(!data_path_.empty() && (((cluster.getLocalShardCount() + cluster.getRemoteShardCount()) < 2) || sharding_key_)),
|
||||
path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(name) + '/'))
|
||||
{
|
||||
createDirectoryMonitors();
|
||||
@ -154,7 +168,7 @@ BlockInputStreams StorageDistributed::read(
|
||||
/// Не имеет смысла на удалённых серверах, так как запрос отправляется обычно с другим user-ом.
|
||||
new_settings.max_concurrent_queries_for_user = 0;
|
||||
|
||||
size_t result_size = (cluster.pools.size() * settings.max_parallel_replicas) + cluster.getLocalNodesNum();
|
||||
size_t result_size = (cluster.getRemoteShardCount() * settings.max_parallel_replicas) + cluster.getLocalShardCount();
|
||||
|
||||
processed_stage = result_size == 1 || settings.distributed_group_by_no_merge
|
||||
? QueryProcessingStage::Complete
|
||||
@ -179,26 +193,31 @@ BlockInputStreams StorageDistributed::read(
|
||||
external_tables = context.getExternalTables();
|
||||
|
||||
/// Цикл по шардам.
|
||||
for (auto & conn_pool : cluster.pools)
|
||||
res.emplace_back(new RemoteBlockInputStream{
|
||||
conn_pool, modified_query, &new_settings, throttler,
|
||||
external_tables, processed_stage, context});
|
||||
|
||||
/// Добавляем запросы к локальному ClickHouse.
|
||||
if (cluster.getLocalNodesNum() > 0)
|
||||
for (const auto & shard_info : cluster.getShardsInfo())
|
||||
{
|
||||
DB::Context new_context = context;
|
||||
new_context.setSettings(new_settings);
|
||||
|
||||
for (size_t i = 0; i < cluster.getLocalNodesNum(); ++i)
|
||||
if (shard_info.isLocal())
|
||||
{
|
||||
InterpreterSelectQuery interpreter(modified_query_ast, new_context, processed_stage);
|
||||
/// Добавляем запросы к локальному ClickHouse.
|
||||
|
||||
/** Материализация нужна, так как с удалённых серверов константы приходят материализованными.
|
||||
* Если этого не делать, то в разных потоках будут получаться разные типы (Const и не-Const) столбцов,
|
||||
* а это не разрешено, так как весь код исходит из допущения, что в потоке блоков все типы одинаковые.
|
||||
*/
|
||||
res.emplace_back(new MaterializingBlockInputStream(interpreter.execute().in));
|
||||
DB::Context new_context = context;
|
||||
new_context.setSettings(new_settings);
|
||||
|
||||
for (const auto & address : shard_info.local_addresses)
|
||||
{
|
||||
InterpreterSelectQuery interpreter(modified_query_ast, new_context, processed_stage);
|
||||
|
||||
/** Материализация нужна, так как с удалённых серверов константы приходят материализованными.
|
||||
* Если этого не делать, то в разных потоках будут получаться разные типы (Const и не-Const) столбцов,
|
||||
* а это не разрешено, так как весь код исходит из допущения, что в потоке блоков все типы одинаковые.
|
||||
*/
|
||||
res.emplace_back(new MaterializingBlockInputStream(interpreter.execute().in));
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
res.emplace_back(new RemoteBlockInputStream{
|
||||
shard_info.pool, modified_query, &new_settings, throttler,
|
||||
external_tables, processed_stage, context});
|
||||
}
|
||||
}
|
||||
|
||||
@ -233,6 +252,63 @@ void StorageDistributed::shutdown()
|
||||
directory_monitors.clear();
|
||||
}
|
||||
|
||||
BlockInputStreams StorageDistributed::describe(const Context & context, const Settings & settings)
|
||||
{
|
||||
Settings new_settings = settings;
|
||||
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.limits.max_execution_time);
|
||||
/// Не имеет смысла на удалённых серверах, так как запрос отправляется обычно с другим user-ом.
|
||||
new_settings.max_concurrent_queries_for_user = 0;
|
||||
|
||||
/// Создать запрос DESCRIBE TABLE.
|
||||
|
||||
auto describe_query = new ASTDescribeQuery;
|
||||
describe_query->database = remote_database;
|
||||
describe_query->table = remote_table;
|
||||
|
||||
ASTPtr ast = describe_query;
|
||||
const auto query = queryToString(ast);
|
||||
|
||||
/// Ограничение сетевого трафика, если нужно.
|
||||
ThrottlerPtr throttler;
|
||||
if (settings.limits.max_network_bandwidth || settings.limits.max_network_bytes)
|
||||
throttler.reset(new Throttler(
|
||||
settings.limits.max_network_bandwidth,
|
||||
settings.limits.max_network_bytes,
|
||||
"Limit for bytes to send or receive over network exceeded."));
|
||||
|
||||
BlockInputStreams res;
|
||||
|
||||
/// Цикл по шардам.
|
||||
for (const auto & shard_info : cluster.getShardsInfo())
|
||||
{
|
||||
if (shard_info.isLocal())
|
||||
{
|
||||
/// Добавляем запросы к локальному ClickHouse.
|
||||
|
||||
DB::Context new_context = context;
|
||||
new_context.setSettings(new_settings);
|
||||
|
||||
for (const auto & address : shard_info.local_addresses)
|
||||
{
|
||||
InterpreterDescribeQuery interpreter(ast, new_context);
|
||||
BlockInputStreamPtr stream = new MaterializingBlockInputStream(interpreter.execute().in);
|
||||
stream = new BlockExtraInfoInputStream(stream, toBlockExtraInfo(address));
|
||||
res.emplace_back(stream);
|
||||
}
|
||||
}
|
||||
|
||||
if (shard_info.hasRemoteConnections())
|
||||
{
|
||||
auto stream = new RemoteBlockInputStream{shard_info.pool, query, &new_settings, throttler};
|
||||
stream->reachAllReplicas();
|
||||
stream->appendExtraInfo();
|
||||
res.emplace_back(stream);
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
NameAndTypePair StorageDistributed::getColumn(const String & column_name) const
|
||||
{
|
||||
if (const auto & type = VirtualColumnFactory::tryGetType(column_name))
|
||||
@ -272,7 +348,7 @@ void StorageDistributed::requireDirectoryMonitor(const std::string & name)
|
||||
|
||||
size_t StorageDistributed::getShardCount() const
|
||||
{
|
||||
return cluster.pools.size();
|
||||
return cluster.getRemoteShardCount();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -72,9 +72,9 @@ BlockInputStreams StorageSystemClusters::read(
|
||||
{
|
||||
const std::string cluster_name = entry.first;
|
||||
const Cluster & cluster = entry.second;
|
||||
const auto & addresses = cluster.getShardsInfo();
|
||||
const auto & addresses_with_failover = cluster.getShardsWithFailoverInfo();
|
||||
const auto & shards_info = cluster.shard_info_vec;
|
||||
const auto & addresses = cluster.getShardsAddresses();
|
||||
const auto & addresses_with_failover = cluster.getShardsWithFailoverAddresses();
|
||||
const auto & shards_info = cluster.getShardsInfo();
|
||||
|
||||
if (!addresses.empty())
|
||||
{
|
||||
|
@ -25,7 +25,8 @@ cat "$CONFIG" | sed -r \
|
||||
s/<http_port>[0-9]+/<http_port>8124/;
|
||||
s/<tcp_port>[0-9]+/<tcp_port>9001/;
|
||||
s/<interserver_http_port>[0-9]+/<interserver_http_port>9010/;
|
||||
s/<use_olap_http_server>true/<use_olap_http_server>false/;
|
||||
s/<olap_compatibility>/<!-- <olap_compatibility>/;
|
||||
s/<\/olap_compatibility>/<\/olap_compatibility> -->/;
|
||||
s/users\.xml/users-preprocessed.xml/' > $CONFIG2
|
||||
|
||||
cp ${CONFIG/config/users} .
|
||||
|
@ -1 +1,2 @@
|
||||
1
|
||||
1
|
||||
|
@ -1 +1,2 @@
|
||||
SELECT min(ts = toUInt32(toDateTime(toString(ts)))) FROM (SELECT 1000000000 + 1234 * number AS ts FROM system.numbers LIMIT 1000000);
|
||||
SELECT min(ts = toUInt32(toDateTime(toString(ts)))) FROM (SELECT 10000 + 1234 * number AS ts FROM system.numbers LIMIT 1000000);
|
||||
|
@ -23,21 +23,186 @@
|
||||
1970-01-02 09:00:00 1970-01-02 10:00:00
|
||||
1970-01-02 18:00:00 1970-01-02 18:00:00
|
||||
1970-01-02 01:30:00 1970-01-02 01:30:00
|
||||
2014
|
||||
2014
|
||||
2014
|
||||
2014
|
||||
2014
|
||||
9
|
||||
9
|
||||
9
|
||||
10
|
||||
9
|
||||
30
|
||||
30
|
||||
30
|
||||
1
|
||||
30
|
||||
2
|
||||
2
|
||||
2
|
||||
3
|
||||
2
|
||||
23
|
||||
21
|
||||
20
|
||||
4
|
||||
11
|
||||
50
|
||||
50
|
||||
50
|
||||
50
|
||||
50
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
2014-09-30 23:50:00
|
||||
2014-09-30 23:50:00
|
||||
2014-09-30 23:50:00
|
||||
2014-09-30 23:50:00
|
||||
2014-09-30 23:50:00
|
||||
2014-09-30 23:00:00
|
||||
2014-09-30 23:00:00
|
||||
2014-09-30 23:00:00
|
||||
2014-09-30 23:00:00
|
||||
2014-09-30 23:00:00
|
||||
2014
|
||||
2014
|
||||
2014
|
||||
2014
|
||||
2014
|
||||
24177
|
||||
24177
|
||||
24177
|
||||
24178
|
||||
24177
|
||||
2334
|
||||
2334
|
||||
2334
|
||||
2334
|
||||
2334
|
||||
16343
|
||||
16343
|
||||
16343
|
||||
16344
|
||||
16343
|
||||
392251
|
||||
392251
|
||||
392251
|
||||
392251
|
||||
392251
|
||||
23535110
|
||||
23535110
|
||||
23535110
|
||||
23535110
|
||||
23535110
|
||||
1412106600
|
||||
1412106600
|
||||
1412106600
|
||||
1412106600
|
||||
1412106600
|
||||
2014-09-30
|
||||
2014-09-30
|
||||
2014-09-30
|
||||
2014-10-01
|
||||
2014-09-30
|
||||
2014-09-30
|
||||
2014-09-30
|
||||
2014-09-30
|
||||
2014-10-01
|
||||
2014-09-30
|
||||
2015-07-13 2015-07-01 2015-07-01 1970-01-02 19:30:00
|
||||
2014-12-22 2014-12-01 2014-10-01 1970-01-02 21:00:00
|
||||
2014-12-29 2015-01-01 2015-01-01 1970-01-02 12:00:00
|
||||
2014-09-29 2014-09-01 2014-07-01 1970-01-02 21:50:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00
|
||||
2015 7 15 3
|
||||
2014 12 28 7
|
||||
2015 1 1 4
|
||||
2014 9 30 2
|
||||
2015 3 15 7
|
||||
19 30 0 2015-07-15 13:30:00
|
||||
21 0 0 2014-12-29 00:00:00
|
||||
12 0 0 2015-01-01 12:00:00
|
||||
21 50 0 2014-09-30 23:50:00
|
||||
2 30 0 2015-03-15 13:30:00
|
||||
2015-07-15 13:00:00 2015 24187 2375
|
||||
2014-12-29 00:00:00 2014 24180 2346
|
||||
2015-01-01 12:00:00 2015 24181 2347
|
||||
2014-09-30 23:00:00 2014 24177 2334
|
||||
2015-03-15 13:00:00 2015 24183 2357
|
||||
16631 399154 23949270 1436956200
|
||||
16432 394389 23663340 1419800400
|
||||
16436 394473 23668380 1420102800
|
||||
16343 392251 23535110 1412106600
|
||||
16509 396226 23773590 1426415400
|
||||
2015-07-15 2015-07-15
|
||||
2014-12-28 2014-12-28
|
||||
2015-01-01 2015-01-01
|
||||
2014-09-30 2014-09-30
|
||||
2015-03-15 2015-03-15
|
||||
2014-09-29 2014-09-01 2014-10-01 1970-01-02 11:20:00
|
||||
2014-12-22 2014-12-01 2014-10-01 1970-01-02 13:30:00
|
||||
2014-12-29 2015-01-01 2015-01-01 1970-01-02 01:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00
|
||||
2015-07-13 2015-07-01 2015-07-01 1970-01-02 02:00:00
|
||||
2015 7 15 3
|
||||
2014 12 29 7
|
||||
2015 1 1 4
|
||||
2014 9 1 2
|
||||
2015 3 15 7
|
||||
12 30 0 2015-07-15 13:30:00
|
||||
22 0 0 2014-12-29 00:00:00
|
||||
10 0 0 2015-01-01 12:00:00
|
||||
21 50 0 2014-09-30 23:50:00
|
||||
11 30 0 2015-03-15 13:30:00
|
||||
2015-07-15 13:00:00 2015 24187 2375
|
||||
2014-12-29 00:00:00 2014 24180 2346
|
||||
2015-01-01 12:00:00 2015 24181 2347
|
||||
2014-09-30 23:00:00 2014 24178 2334
|
||||
2015-03-15 13:00:00 2015 24183 2357
|
||||
16631 399154 23949270 1436956200
|
||||
16432 394389 23663340 1419800400
|
||||
16436 394473 23668380 1420102800
|
||||
16343 392251 23535110 1412106600
|
||||
16509 396226 23773590 1426415400
|
||||
2015-07-15 2015-07-15
|
||||
2014-12-28 2014-12-28
|
||||
2015-01-01 2015-01-01
|
||||
2014-09-30 2014-09-30
|
||||
2015-03-15 2015-03-15
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 19:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 10:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 13:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 11:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00
|
||||
2015 3 15 7
|
||||
2015 3 15 7
|
||||
2015 3 15 7
|
||||
2015 3 15 7
|
||||
2015 3 15 7
|
||||
19 30 0 2015-03-15 13:30:00
|
||||
10 30 0 2015-03-15 13:30:00
|
||||
13 30 0 2015-03-15 13:30:00
|
||||
11 30 0 2015-03-15 13:30:00
|
||||
2 30 0 2015-03-15 13:30:00
|
||||
2015-03-15 13:00:00 2015 24183 2357
|
||||
2015-03-15 13:00:00 2015 24183 2357
|
||||
2015-03-15 13:00:00 2015 24183 2357
|
||||
2015-03-15 13:00:00 2015 24183 2357
|
||||
2015-03-15 13:00:00 2015 24183 2357
|
||||
16509 396226 23773590 1426415400
|
||||
16509 396226 23773590 1426415400
|
||||
16509 396226 23773590 1426415400
|
||||
16509 396226 23773590 1426415400
|
||||
16509 396226 23773590 1426415400
|
||||
2015-03-15 2015-03-15
|
||||
2015-03-15 2015-03-15
|
||||
2015-03-15 2015-03-15
|
||||
2015-03-15 2015-03-15
|
||||
2015-03-15 2015-03-15
|
||||
2015-07-15 13:30:00
|
||||
2015-07-15 12:30:00
|
||||
2015-07-15 11:30:00
|
||||
|
@ -46,6 +46,148 @@ SELECT toTime(toDateTime(1420102800), 'Europe/London'), toTime(toDateTime(142831
|
||||
SELECT toTime(toDateTime(1420102800), 'Asia/Tokyo'), toTime(toDateTime(1428310800), 'Asia/Tokyo');
|
||||
SELECT toTime(toDateTime(1420102800), 'Pacific/Pitcairn'), toTime(toDateTime(1428310800), 'Pacific/Pitcairn');
|
||||
|
||||
/* toYear */
|
||||
|
||||
SELECT toYear(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toYear(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toYear(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toYear(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toYear(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toMonth */
|
||||
|
||||
SELECT toMonth(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toMonth(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toMonth(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toMonth(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toMonth(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toDayOfMonth */
|
||||
|
||||
SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toDayOfMonth(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toDayOfMonth(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toDayOfWeek */
|
||||
|
||||
SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toDayOfWeek(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toDayOfWeek(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toHour */
|
||||
|
||||
SELECT toHour(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toHour(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toHour(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toHour(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toHour(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toMinute */
|
||||
|
||||
SELECT toMinute(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toMinute(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toMinute(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toMinute(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toMinute(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toSecond */
|
||||
|
||||
SELECT toSecond(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toSecond(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toSecond(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toSecond(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toSecond(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toStartOfMinute */
|
||||
|
||||
SELECT toStartOfMinute(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toStartOfMinute(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toStartOfMinute(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toStartOfMinute(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toStartOfMinute(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toStartOfHour */
|
||||
|
||||
SELECT toStartOfHour(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toStartOfHour(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toStartOfHour(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toStartOfHour(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toStartOfHour(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toRelativeYearNum */
|
||||
|
||||
SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toRelativeYearNum(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toRelativeYearNum(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toRelativeMonthNum */
|
||||
|
||||
SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toRelativeMonthNum(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toRelativeMonthNum(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toRelativeWeekNum */
|
||||
|
||||
SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toRelativeWeekNum(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toRelativeWeekNum(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toRelativeDayNum */
|
||||
|
||||
SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toRelativeDayNum(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toRelativeDayNum(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toRelativeHourNum */
|
||||
|
||||
SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toRelativeHourNum(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toRelativeHourNum(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toRelativeMinuteNum */
|
||||
|
||||
SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toRelativeSecondNum */
|
||||
|
||||
SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toRelativeSecondNum(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toRelativeSecondNum(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toDate */
|
||||
|
||||
SELECT toDate(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toDate(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toDate(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toDate(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toDate(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
SELECT toDate(1412106600, 'Europe/Moscow');
|
||||
SELECT toDate(1412106600, 'Europe/Paris');
|
||||
SELECT toDate(1412106600, 'Europe/London');
|
||||
SELECT toDate(1412106600, 'Asia/Tokyo');
|
||||
SELECT toDate(1412106600, 'Pacific/Pitcairn');
|
||||
|
||||
DROP TABLE IF EXISTS foo;
|
||||
CREATE TABLE foo(x Int32, y String) ENGINE=Memory;
|
||||
INSERT INTO foo(x, y) VALUES(1420102800, 'Europe/Moscow');
|
||||
@ -55,8 +197,25 @@ INSERT INTO foo(x, y) VALUES(1436956200, 'Asia/Tokyo');
|
||||
INSERT INTO foo(x, y) VALUES(1426415400, 'Pacific/Pitcairn');
|
||||
|
||||
SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toYear(toDateTime(x), y), toMonth(toDateTime(x), y), toDayOfMonth(toDateTime(x), y), toDayOfWeek(toDateTime(x), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toHour(toDateTime(x), y), toMinute(toDateTime(x), y), toSecond(toDateTime(x), y), toStartOfMinute(toDateTime(x), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toStartOfHour(toDateTime(x), y), toRelativeYearNum(toDateTime(x), y), toRelativeMonthNum(toDateTime(x), y), toRelativeWeekNum(toDateTime(x), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toRelativeDayNum(toDateTime(x), y), toRelativeHourNum(toDateTime(x), y), toRelativeMinuteNum(toDateTime(x), y), toRelativeSecondNum(toDateTime(x), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toDate(toDateTime(x), y), toDate(x, y) FROM foo ORDER BY y ASC;
|
||||
|
||||
SELECT toMonday(toDateTime(x), 'Europe/Paris'), toStartOfMonth(toDateTime(x), 'Europe/London'), toStartOfQuarter(toDateTime(x), 'Asia/Tokyo'), toTime(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY x ASC;
|
||||
SELECT toYear(toDateTime(x), 'Europe/Paris'), toMonth(toDateTime(x), 'Europe/London'), toDayOfMonth(toDateTime(x), 'Asia/Tokyo'), toDayOfWeek(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC;
|
||||
SELECT toHour(toDateTime(x), 'Europe/Paris'), toMinute(toDateTime(x), 'Europe/London'), toSecond(toDateTime(x), 'Asia/Tokyo'), toStartOfMinute(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC;
|
||||
SELECT toStartOfHour(toDateTime(x), 'Europe/Paris'), toRelativeYearNum(toDateTime(x), 'Europe/London'), toRelativeMonthNum(toDateTime(x), 'Asia/Tokyo'), toRelativeWeekNum(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC;
|
||||
SELECT toRelativeDayNum(toDateTime(x), 'Europe/Paris'), toRelativeHourNum(toDateTime(x), 'Europe/London'), toRelativeMinuteNum(toDateTime(x), 'Asia/Tokyo'), toRelativeSecondNum(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC;
|
||||
SELECT toDate(toDateTime(x), 'Europe/Paris'), toDate(x, 'Europe/Paris') FROM foo ORDER BY y ASC;
|
||||
|
||||
SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toYear(toDateTime(1426415400), y), toMonth(toDateTime(1426415400), y), toDayOfMonth(toDateTime(1426415400), y), toDayOfWeek(toDateTime(1426415400), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toHour(toDateTime(1426415400), y), toMinute(toDateTime(1426415400), y), toSecond(toDateTime(1426415400), y), toStartOfMinute(toDateTime(1426415400), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toStartOfHour(toDateTime(1426415400), y), toRelativeYearNum(toDateTime(1426415400), y), toRelativeMonthNum(toDateTime(1426415400), y), toRelativeWeekNum(toDateTime(1426415400), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toRelativeDayNum(toDateTime(1426415400), y), toRelativeHourNum(toDateTime(1426415400), y), toRelativeMinuteNum(toDateTime(1426415400), y), toRelativeSecondNum(toDateTime(1426415400), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toDate(toDateTime(1426415400), y), toDate(1426415400, y) FROM foo ORDER BY y ASC;
|
||||
|
||||
/* toString */
|
||||
|
||||
|
@ -2,9 +2,9 @@
|
||||
1
|
||||
1
|
||||
|
||||
0
|
||||
1
|
||||
|
||||
0
|
||||
1
|
||||
|
||||
1
|
||||
1
|
||||
|
@ -0,0 +1,2 @@
|
||||
20
|
||||
20
|
@ -0,0 +1,6 @@
|
||||
DROP TABLE IF EXISTS test.storage;
|
||||
CREATE TABLE test.storage(UserID UInt64) ENGINE=Memory;
|
||||
INSERT INTO test.storage(UserID) values (6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(402895971392036118)(402895971392036118)(402895971392036118);
|
||||
|
||||
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM remote('127.0.0.{1,2}', test.storage))) FROM remote('127.0.0.{1,2}', test.storage);
|
||||
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM test.storage)) FROM remote('127.0.0.{1,2}', test.storage);
|
@ -0,0 +1,2 @@
|
||||
1 0 0
|
||||
2015-01-01 2015-01-01 01:02:03 111 123 456 789 456 9434005089510819894 9434005089510819894
|
@ -0,0 +1,23 @@
|
||||
DROP TABLE IF EXISTS test.defaults;
|
||||
CREATE TABLE test.defaults (a UInt8, b DEFAULT 0, c DEFAULT identity(b)) ENGINE = Memory;
|
||||
INSERT INTO test.defaults (a) VALUES (1);
|
||||
SELECT * FROM test.defaults;
|
||||
DROP TABLE test.defaults;
|
||||
|
||||
DROP TABLE IF EXISTS test.elog_cut;
|
||||
CREATE TABLE test.elog_cut
|
||||
(
|
||||
date Date DEFAULT toDate(uts),
|
||||
uts DateTime,
|
||||
pr UInt64,
|
||||
ya_uid UInt64,
|
||||
adf_uid UInt64,
|
||||
owner_id UInt32,
|
||||
eff_uid UInt64 DEFAULT if(adf_uid != 0, adf_uid, ya_uid),
|
||||
page_session UInt64 DEFAULT cityHash64(eff_uid, pr),
|
||||
sample_key UInt64 ALIAS page_session
|
||||
) ENGINE = MergeTree(date, cityHash64(adf_uid, ya_uid, pr), (owner_id, date, cityHash64(adf_uid, ya_uid, pr)), 8192);
|
||||
|
||||
INSERT INTO test.elog_cut (uts, pr, ya_uid, adf_uid, owner_id) VALUES ('2015-01-01 01:02:03', 111, 123, 456, 789);
|
||||
SELECT date, uts, pr, ya_uid, adf_uid, owner_id, eff_uid, page_session, sample_key FROM test.elog_cut;
|
||||
DROP TABLE test.elog_cut;
|
@ -0,0 +1,13 @@
|
||||
0 (0,'2015-01-01')
|
||||
1 (1,'2015-01-02')
|
||||
2 (2,'2015-01-03')
|
||||
3 (3,'2015-01-04')
|
||||
4 (4,'2015-01-05')
|
||||
5 (5,'2015-01-06')
|
||||
6 (6,'2015-01-07')
|
||||
7 (7,'2015-01-08')
|
||||
8 (8,'2015-01-09')
|
||||
9 (9,'2015-01-10')
|
||||
|
||||
0 (0,'2015-01-01')
|
||||
9 (9,'2015-01-10')
|
1
dbms/tests/queries/0_stateless/00254_tuple_extremes.sql
Normal file
1
dbms/tests/queries/0_stateless/00254_tuple_extremes.sql
Normal file
@ -0,0 +1 @@
|
||||
SELECT number, (number, toDate('2015-01-01') + number) FROM system.numbers LIMIT 10 SETTINGS extremes = 1;
|
@ -0,0 +1,65 @@
|
||||
HelloWorld
|
||||
HelloWorld
|
||||
Hello, World
|
||||
Hello, World
|
||||
|
||||
|
||||
0
|
||||
01
|
||||
012
|
||||
0123
|
||||
01234
|
||||
012345
|
||||
0123456
|
||||
01234567
|
||||
012345678
|
||||
|
||||
0
|
||||
01
|
||||
012
|
||||
0123
|
||||
01234
|
||||
012345
|
||||
0123456
|
||||
01234567
|
||||
012345678
|
||||
|
||||
0
|
||||
0,1
|
||||
0,1,2
|
||||
0,1,2,3
|
||||
0,1,2,3,4
|
||||
0,1,2,3,4,5
|
||||
0,1,2,3,4,5,6
|
||||
0,1,2,3,4,5,6,7
|
||||
0,1,2,3,4,5,6,7,8
|
||||
|
||||
yandex
|
||||
yandex google
|
||||
yandex google test
|
||||
yandex google test 123
|
||||
yandex google test 123
|
||||
yandex google test 123 hello
|
||||
yandex google test 123 hello world
|
||||
yandex google test 123 hello world goodbye
|
||||
yandex google test 123 hello world goodbye xyz
|
||||
yandex google test 123 hello world goodbye xyz yandex
|
||||
yandex google test 123 hello world goodbye xyz yandex google
|
||||
yandex google test 123 hello world goodbye xyz yandex google test
|
||||
yandex google test 123 hello world goodbye xyz yandex google test 123
|
||||
yandex google test 123 hello world goodbye xyz yandex google test 123
|
||||
yandex google test 123 hello world goodbye xyz yandex google test 123 hello
|
||||
yandex google test 123 hello world goodbye xyz yandex google test 123 hello world
|
||||
yandex google test 123 hello world goodbye xyz yandex google test 123 hello world goodbye
|
||||
yandex google test 123 hello world goodbye xyz yandex google test 123 hello world goodbye xyz
|
||||
yandex google test 123 hello world goodbye xyz yandex google test 123 hello world goodbye xyz yandex
|
||||
|
||||
0
|
||||
01
|
||||
012
|
||||
|
||||
0
|
||||
01
|
||||
012
|
||||
|
||||
0
|
10
dbms/tests/queries/0_stateless/00255_array_concat_string.sql
Normal file
10
dbms/tests/queries/0_stateless/00255_array_concat_string.sql
Normal file
@ -0,0 +1,10 @@
|
||||
SELECT arrayStringConcat(['Hello', 'World']);
|
||||
SELECT arrayStringConcat(materialize(['Hello', 'World']));
|
||||
SELECT arrayStringConcat(['Hello', 'World'], ', ');
|
||||
SELECT arrayStringConcat(materialize(['Hello', 'World']), ', ');
|
||||
SELECT arrayStringConcat(emptyArrayString());
|
||||
SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||
SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number)), '') FROM system.numbers LIMIT 10;
|
||||
SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number)), ',') FROM system.numbers LIMIT 10;
|
||||
SELECT arrayStringConcat(arrayMap(x -> transform(x, [0, 1, 2, 3, 4, 5, 6, 7, 8], ['yandex', 'google', 'test', '123', '', 'hello', 'world', 'goodbye', 'xyz'], ''), arrayMap(x -> x % 9, range(number))), ' ') FROM system.numbers LIMIT 20;
|
||||
SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number % 4))) FROM system.numbers LIMIT 10;
|
44
dbms/tests/queries/0_stateless/00256_reverse.reference
Normal file
44
dbms/tests/queries/0_stateless/00256_reverse.reference
Normal file
@ -0,0 +1,44 @@
|
||||
olleH
|
||||
olleH
|
||||
1
|
||||
01
|
||||
001
|
||||
0001
|
||||
00001
|
||||
000001
|
||||
0000001
|
||||
00000001
|
||||
000000001
|
||||
0000000001
|
||||
['World','Hello']
|
||||
['World','Hello']
|
||||
[]
|
||||
[0]
|
||||
[1,0]
|
||||
[2,1,0]
|
||||
[3,2,1,0]
|
||||
[4,3,2,1,0]
|
||||
[5,4,3,2,1,0]
|
||||
[6,5,4,3,2,1,0]
|
||||
[7,6,5,4,3,2,1,0]
|
||||
[8,7,6,5,4,3,2,1,0]
|
||||
[]
|
||||
['1']
|
||||
['10','1']
|
||||
['100','10','1']
|
||||
['1000','100','10','1']
|
||||
['10000','1000','100','10','1']
|
||||
['100000','10000','1000','100','10','1']
|
||||
['1000000','100000','10000','1000','100','10','1']
|
||||
['10000000','1000000','100000','10000','1000','100','10','1']
|
||||
['100000000','10000000','1000000','100000','10000','1000','100','10','1']
|
||||
\0\0\0\0\0\0\0\0\01
|
||||
\0\0\0\0\0\0\0\001
|
||||
\0\0\0\0\0\0\0001
|
||||
\0\0\0\0\0\00001
|
||||
\0\0\0\0\000001
|
||||
\0\0\0\0000001
|
||||
\0\0\00000001
|
||||
\0\000000001
|
||||
\0000000001
|
||||
0000000001
|
9
dbms/tests/queries/0_stateless/00256_reverse.sql
Normal file
9
dbms/tests/queries/0_stateless/00256_reverse.sql
Normal file
@ -0,0 +1,9 @@
|
||||
SELECT reverse('Hello');
|
||||
SELECT reverse(materialize('Hello'));
|
||||
SELECT reverse(toString(exp10(number))) FROM system.numbers LIMIT 10;
|
||||
|
||||
SELECT reverse(['Hello', 'World']);
|
||||
SELECT reverse(materialize(['Hello', 'World']));
|
||||
SELECT reverse(range(number)) FROM system.numbers LIMIT 10;
|
||||
SELECT reverse(arrayMap(x -> toString(exp10(x)), range(number))) FROM system.numbers LIMIT 10;
|
||||
SELECT reverse(toFixedString(toString(exp10(number)), 10)) FROM system.numbers LIMIT 10;
|
@ -0,0 +1,29 @@
|
||||
40
|
||||
41
|
||||
2 42
|
||||
43
|
||||
11
|
||||
40
|
||||
|
||||
40
|
||||
41
|
||||
|
||||
41
|
||||
2 42
|
||||
|
||||
2 42
|
||||
43
|
||||
|
||||
43
|
||||
11
|
||||
|
||||
11
|
||||
11
|
||||
|
||||
11
|
||||
1
|
||||
|
||||
1
|
||||
2
|
||||
|
||||
2
|
@ -0,0 +1,15 @@
|
||||
select 40 as z from (select * from system.numbers limit 3) group by z;
|
||||
select 41 as z from remote('127.0.0.{1,2}', system.one) group by z;
|
||||
select count(), 42 AS z from remote('127.0.0.{1,2}', system.one) group by z;
|
||||
select 43 AS z from remote('127.0.0.{1,2}', system.one) group by 42, 43, 44;
|
||||
select 11 AS z from (SELECT 2 UNION ALL SELECT 3) group by 42, 43, 44;
|
||||
|
||||
select 40 as z from (select * from system.numbers limit 3) group by z WITH TOTALS;
|
||||
select 41 as z from remote('127.0.0.{1,2}', system.one) group by z WITH TOTALS;
|
||||
select count(), 42 AS z from remote('127.0.0.{1,2}', system.one) group by z WITH TOTALS;
|
||||
select 43 AS z from remote('127.0.0.{1,2}', system.one) group by 42, 43, 44 WITH TOTALS;
|
||||
select 11 AS z from (SELECT 1 UNION ALL SELECT 2) group by 42, 43, 44 WITH TOTALS;
|
||||
select 11 AS z from (SELECT 2 UNION ALL SELECT 3) group by 42, 43, 44 WITH TOTALS;
|
||||
|
||||
SELECT count() WITH TOTALS;
|
||||
SELECT count() FROM remote('127.0.0.{1,2}', system.one) WITH TOTALS;
|
23
libs/libcommon/include/ext/collection_cast.hpp
Normal file
23
libs/libcommon/include/ext/collection_cast.hpp
Normal file
@ -0,0 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
|
||||
namespace ext
|
||||
{
|
||||
/** \brief Returns collection of specified container-type.
|
||||
* Retains stored value_type, constructs resulting collection using iterator range. */
|
||||
template <template <typename...> class ResultCollection, typename Collection>
|
||||
auto collection_cast(const Collection & collection)
|
||||
{
|
||||
using value_type = typename Collection::value_type;
|
||||
|
||||
return ResultCollection<value_type>(std::begin(collection), std::end(collection));
|
||||
};
|
||||
|
||||
/** \brief Returns collection of specified type.
|
||||
* Performs implicit conversion of between source and result value_type, if available and required. */
|
||||
template <typename ResultCollection, typename Collection>
|
||||
auto collection_cast(const Collection & collection)
|
||||
{
|
||||
return ResultCollection(std::begin(collection), std::end(collection));
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user