This commit is contained in:
Roman Peshkurov 2015-10-28 16:33:01 +03:00
commit eb607dc611
104 changed files with 1006619 additions and 1497 deletions

View File

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

View File

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

View File

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

View File

@ -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;
/// Запрос выполняется параллельно на нескольких репликах.

View File

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

View File

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

View 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");
};
}

View File

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

View File

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

View File

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

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

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

View File

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

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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.
*/

View File

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

View File

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

View File

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

View File

@ -13,6 +13,9 @@ public:
InterpreterCheckQuery(ASTPtr query_ptr_, Context & context_);
BlockIO execute() override;
private:
Block getSampleBlock() const;
private:
ASTPtr query_ptr;
Context context;

View File

@ -166,6 +166,8 @@ struct Settings
/** Для запросов SELECT из реплицируемой таблицы, кидать исключение, если на реплике нет куска, записанного с кворумом; \
* не читать куски, которые ещё не были записаны с кворумом. */ \
M(SettingUInt64, select_sequential_consistency, 0) \
/** Максимальное количество различных шардов и максимальное количество реплик одного шарда в функции remote. */ \
M(SettingUInt64, table_function_remote_max_addresses, 1000) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;

View File

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

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

@ -10,6 +10,7 @@ void registerFunctionsStringArray(FunctionFactory & factory)
factory.registerFunction<FunctionAlphaTokens>();
factory.registerFunction<FunctionSplitByChar>();
factory.registerFunction<FunctionSplitByString>();
factory.registerFunction<FunctionArrayStringConcat>();
}
}

View File

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

File diff suppressed because it is too large Load Diff

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

File diff suppressed because it is too large Load Diff

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -2,9 +2,9 @@
1
1
0
1
0
1
1
1

View File

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

View File

@ -0,0 +1,2 @@
1 0 0
2015-01-01 2015-01-01 01:02:03 111 123 456 789 456 9434005089510819894 9434005089510819894

View File

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

View File

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

View File

@ -0,0 +1 @@
SELECT number, (number, toDate('2015-01-01') + number) FROM system.numbers LIMIT 10 SETTINGS extremes = 1;

View File

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

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

View 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

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

View File

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

View File

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

View 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