This commit is contained in:
Evgeniy Gatov 2015-03-09 01:22:07 +03:00
commit db00a42585
62 changed files with 1532 additions and 1101 deletions

View File

@ -3,41 +3,31 @@
#include <DB/IO/WriteHelpers.h> #include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h> #include <DB/IO/ReadHelpers.h>
#include <DB/AggregateFunctions/IAggregateFunction.h> #include <DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h>
namespace DB namespace DB
{ {
struct AggregateFunctionArgMinTraits /// Возможные значения параметров шаблонов см. в AggregateFunctionsMinMaxAny.h
{ template <typename ResultData, typename ValueData>
static bool better(const Field & lhs, const Field & rhs) { return lhs < rhs; }
static String name() { return "argMin"; }
};
struct AggregateFunctionArgMaxTraits
{
static bool better(const Field & lhs, const Field & rhs) { return lhs > rhs; }
static String name() { return "argMax"; }
};
struct AggregateFunctionsArgMinMaxData struct AggregateFunctionsArgMinMaxData
{ {
Field result; // аргумент, при котором достигается минимальное/максимальное значение value. ResultData result; // аргумент, при котором достигается минимальное/максимальное значение value.
Field value; // значение, для которого считается минимум/максимум. ValueData value; // значение, для которого считается минимум/максимум.
}; };
/// Возвращает первое попавшееся значение arg для минимального/максимального value. Пример: argMax(arg, value). /// Возвращает первое попавшееся значение arg для минимального/максимального value. Пример: argMax(arg, value).
template <typename Traits> template <typename Data>
class AggregateFunctionsArgMinMax final : public IAggregateFunctionHelper<AggregateFunctionsArgMinMaxData> class AggregateFunctionsArgMinMax final : public IAggregateFunctionHelper<Data>
{ {
private: private:
DataTypePtr type_res; DataTypePtr type_res;
DataTypePtr type_val; DataTypePtr type_val;
public: public:
String getName() const { return Traits::name(); } String getName() const { return (0 == strcmp(decltype(Data::value)::name(), "min")) ? "argMin" : "argMax"; }
DataTypePtr getReturnType() const DataTypePtr getReturnType() const
{ {
@ -55,105 +45,37 @@ public:
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
{ {
Field result; if (this->data(place).value.changeIfBetter(*columns[1], row_num))
Field value; this->data(place).result.change(*columns[0], row_num);
columns[0]->get(row_num, result);
columns[1]->get(row_num, value);
Data & d = data(place);
if (!d.value.isNull())
{
if (Traits::better(value, d.value))
{
d.result = result;
d.value = value;
}
}
else
{
d.result = result;
d.value = value;
}
} }
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
{ {
Data & d = data(place); if (this->data(place).value.changeIfBetter(this->data(rhs).value))
const Data & d_rhs = data(rhs); this->data(place).result.change(this->data(rhs).result);
if (!d.value.isNull())
{
if (Traits::better(d_rhs.value, d.value))
{
d.result = d_rhs.result;
d.value = d_rhs.value;
}
}
else
{
d.result = d_rhs.result;
d.value = d_rhs.value;
}
} }
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
{ {
const Data & d = data(place); this->data(place).result.write(buf, *type_res.get());
this->data(place).value.write(buf, *type_val.get());
if (unlikely(d.result.isNull()))
{
writeBinary(false, buf);
}
else
{
writeBinary(true, buf);
type_res->serializeBinary(d.result, buf);
type_val->serializeBinary(d.value, buf);
}
} }
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
{ {
Data & d = data(place); Data rhs; /// Для строчек не очень оптимально, так как может делаться одна лишняя аллокация.
bool is_not_null = false; rhs.result.read(buf, *type_res.get());
readBinary(is_not_null, buf); rhs.value.read(buf, *type_val.get());
if (is_not_null) if (this->data(place).value.changeIfBetter(rhs.value))
{ this->data(place).result.change(rhs.result);
if (!d.value.isNull())
{
Field result_;
Field value_;
type_res->deserializeBinary(result_, buf);
type_val->deserializeBinary(value_, buf);
if (Traits::better(value_, d.value))
{
d.result = result_;
d.value = value_;
}
}
else
{
type_res->deserializeBinary(d.result, buf);
type_val->deserializeBinary(d.value, buf);
}
}
} }
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
{ {
if (unlikely(data(place).value.isNull())) this->data(place).result.insertResultInto(to);
to.insertDefault();
else
to.insert(data(place).result);
} }
}; };
typedef AggregateFunctionsArgMinMax<AggregateFunctionArgMinTraits> AggregateFunctionArgMin;
typedef AggregateFunctionsArgMinMax<AggregateFunctionArgMaxTraits> AggregateFunctionArgMax;
} }

View File

@ -67,40 +67,70 @@ struct SingleValueDataFixed
value = to.value; value = to.value;
} }
void changeFirstTime(const IColumn & column, size_t row_num) bool changeFirstTime(const IColumn & column, size_t row_num)
{ {
if (!has()) if (!has())
{
change(column, row_num); change(column, row_num);
return true;
}
else
return false;
} }
void changeFirstTime(const Self & to) bool changeFirstTime(const Self & to)
{ {
if (!has()) if (!has())
{
change(to); change(to);
return true;
}
else
return false;
} }
void changeIfLess(const IColumn & column, size_t row_num) bool changeIfLess(const IColumn & column, size_t row_num)
{ {
if (!has() || static_cast<const ColumnVector<T> &>(column).getData()[row_num] < value) if (!has() || static_cast<const ColumnVector<T> &>(column).getData()[row_num] < value)
{
change(column, row_num); change(column, row_num);
return true;
}
else
return false;
} }
void changeIfLess(const Self & to) bool changeIfLess(const Self & to)
{ {
if (to.has() && (!has() || to.value < value)) if (to.has() && (!has() || to.value < value))
{
change(to); change(to);
return true;
}
else
return false;
} }
void changeIfGreater(const IColumn & column, size_t row_num) bool changeIfGreater(const IColumn & column, size_t row_num)
{ {
if (!has() || static_cast<const ColumnVector<T> &>(column).getData()[row_num] > value) if (!has() || static_cast<const ColumnVector<T> &>(column).getData()[row_num] > value)
{
change(column, row_num); change(column, row_num);
return true;
}
else
return false;
} }
void changeIfGreater(const Self & to) bool changeIfGreater(const Self & to)
{ {
if (to.has() && (!has() || to.value > value)) if (to.has() && (!has() || to.value > value))
{
change(to); change(to);
return true;
}
else
return false;
} }
}; };
@ -238,40 +268,70 @@ struct __attribute__((__packed__)) SingleValueDataString
changeImpl(to.getStringRef()); changeImpl(to.getStringRef());
} }
void changeFirstTime(const IColumn & column, size_t row_num) bool changeFirstTime(const IColumn & column, size_t row_num)
{ {
if (!has()) if (!has())
{
change(column, row_num); change(column, row_num);
return true;
}
else
return false;
} }
void changeFirstTime(const Self & to) bool changeFirstTime(const Self & to)
{ {
if (!has()) if (!has())
{
change(to); change(to);
return true;
}
else
return false;
} }
void changeIfLess(const IColumn & column, size_t row_num) bool changeIfLess(const IColumn & column, size_t row_num)
{ {
if (!has() || static_cast<const ColumnString &>(column).getDataAtWithTerminatingZero(row_num) < getStringRef()) if (!has() || static_cast<const ColumnString &>(column).getDataAtWithTerminatingZero(row_num) < getStringRef())
{
change(column, row_num); change(column, row_num);
return true;
}
else
return false;
} }
void changeIfLess(const Self & to) bool changeIfLess(const Self & to)
{ {
if (to.has() && (!has() || to.getStringRef() < getStringRef())) if (to.has() && (!has() || to.getStringRef() < getStringRef()))
{
change(to); change(to);
return true;
}
else
return false;
} }
void changeIfGreater(const IColumn & column, size_t row_num) bool changeIfGreater(const IColumn & column, size_t row_num)
{ {
if (!has() || static_cast<const ColumnString &>(column).getDataAtWithTerminatingZero(row_num) > getStringRef()) if (!has() || static_cast<const ColumnString &>(column).getDataAtWithTerminatingZero(row_num) > getStringRef())
{
change(column, row_num); change(column, row_num);
return true;
}
else
return false;
} }
void changeIfGreater(const Self & to) bool changeIfGreater(const Self & to)
{ {
if (to.has() && (!has() || to.getStringRef() > getStringRef())) if (to.has() && (!has() || to.getStringRef() > getStringRef()))
{
change(to); change(to);
return true;
}
else
return false;
} }
}; };
@ -326,54 +386,90 @@ struct SingleValueDataGeneric
value = to.value; value = to.value;
} }
void changeFirstTime(const IColumn & column, size_t row_num) bool changeFirstTime(const IColumn & column, size_t row_num)
{ {
if (!has()) if (!has())
{
change(column, row_num); change(column, row_num);
return true;
}
else
return false;
} }
void changeFirstTime(const Self & to) bool changeFirstTime(const Self & to)
{ {
if (!has()) if (!has())
{
change(to); change(to);
return true;
}
else
return false;
} }
void changeIfLess(const IColumn & column, size_t row_num) bool changeIfLess(const IColumn & column, size_t row_num)
{ {
if (!has()) if (!has())
{
change(column, row_num); change(column, row_num);
return true;
}
else else
{ {
Field new_value; Field new_value;
column.get(row_num, new_value); column.get(row_num, new_value);
if (new_value < value) if (new_value < value)
{
value = new_value; value = new_value;
return true;
}
else
return false;
} }
} }
void changeIfLess(const Self & to) bool changeIfLess(const Self & to)
{ {
if (to.has() && (!has() || to.value < value)) if (to.has() && (!has() || to.value < value))
{
change(to); change(to);
return true;
}
else
return false;
} }
void changeIfGreater(const IColumn & column, size_t row_num) bool changeIfGreater(const IColumn & column, size_t row_num)
{ {
if (!has()) if (!has())
{
change(column, row_num); change(column, row_num);
return true;
}
else else
{ {
Field new_value; Field new_value;
column.get(row_num, new_value); column.get(row_num, new_value);
if (new_value > value) if (new_value > value)
{
value = new_value; value = new_value;
return true;
}
else
return false;
} }
} }
void changeIfGreater(const Self & to) bool changeIfGreater(const Self & to)
{ {
if (to.has() && (!has() || to.value > value)) if (to.has() && (!has() || to.value > value))
{
change(to); change(to);
return true;
}
else
return false;
} }
}; };
@ -388,8 +484,8 @@ struct AggregateFunctionMinData : Data
{ {
typedef AggregateFunctionMinData<Data> Self; typedef AggregateFunctionMinData<Data> Self;
void changeIfBetter(const IColumn & column, size_t row_num) { this->changeIfLess(column, row_num); } bool changeIfBetter(const IColumn & column, size_t row_num) { return this->changeIfLess(column, row_num); }
void changeIfBetter(const Self & to) { this->changeIfLess(to); } bool changeIfBetter(const Self & to) { return this->changeIfLess(to); }
static const char * name() { return "min"; } static const char * name() { return "min"; }
}; };
@ -399,8 +495,8 @@ struct AggregateFunctionMaxData : Data
{ {
typedef AggregateFunctionMaxData<Data> Self; typedef AggregateFunctionMaxData<Data> Self;
void changeIfBetter(const IColumn & column, size_t row_num) { this->changeIfGreater(column, row_num); } bool changeIfBetter(const IColumn & column, size_t row_num) { return this->changeIfGreater(column, row_num); }
void changeIfBetter(const Self & to) { this->changeIfGreater(to); } bool changeIfBetter(const Self & to) { return this->changeIfGreater(to); }
static const char * name() { return "max"; } static const char * name() { return "max"; }
}; };
@ -410,8 +506,8 @@ struct AggregateFunctionAnyData : Data
{ {
typedef AggregateFunctionAnyData<Data> Self; typedef AggregateFunctionAnyData<Data> Self;
void changeIfBetter(const IColumn & column, size_t row_num) { this->changeFirstTime(column, row_num); } bool changeIfBetter(const IColumn & column, size_t row_num) { return this->changeFirstTime(column, row_num); }
void changeIfBetter(const Self & to) { this->changeFirstTime(to); } bool changeIfBetter(const Self & to) { return this->changeFirstTime(to); }
static const char * name() { return "any"; } static const char * name() { return "any"; }
}; };
@ -421,8 +517,8 @@ struct AggregateFunctionAnyLastData : Data
{ {
typedef AggregateFunctionAnyLastData<Data> Self; typedef AggregateFunctionAnyLastData<Data> Self;
void changeIfBetter(const IColumn & column, size_t row_num) { this->change(column, row_num); } bool changeIfBetter(const IColumn & column, size_t row_num) { this->change(column, row_num); return true; }
void changeIfBetter(const Self & to) { this->change(to); } bool changeIfBetter(const Self & to) { this->change(to); return true; }
static const char * name() { return "anyLast"; } static const char * name() { return "anyLast"; }
}; };

View File

@ -39,7 +39,7 @@ public:
/// Разорвать все действующие соединения. /// Разорвать все действующие соединения.
void disconnect(); void disconnect();
/// Отменить запросы к репликам /// Отправить на реплики просьбу отменить выполнение запроса
void sendCancel(); void sendCancel();
/** На каждой реплике читать и пропускать все пакеты до EndOfStream или Exception. /** На каждой реплике читать и пропускать все пакеты до EndOfStream или Exception.

View File

@ -156,8 +156,9 @@ typedef ColumnConst<Array> ColumnConstArray;
template <typename T> ColumnPtr ColumnConst<T>::convertToFullColumn() const template <typename T> ColumnPtr ColumnConst<T>::convertToFullColumn() const
{ {
ColumnVector<T> * res = new ColumnVector<T>; ColumnVector<T> * res_ = new ColumnVector<T>;
res->getData().assign(s, data); ColumnPtr res = res_;
res_->getData().assign(s, data);
return res; return res;
} }

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <DB/Columns/IColumn.h> #include <DB/Columns/IColumn.h>
#include <DB/Interpreters/Set.h>
namespace DB namespace DB

View File

@ -71,7 +71,7 @@ public:
this->emplace(x, it, inserted); this->emplace(x, it, inserted);
if (inserted) if (inserted)
new(&it->second) mapped_type(); /// В отличие от HashMap, всегда инициализируем значение. new(&it->second) mapped_type();
return it->second; return it->second;
} }

View File

@ -137,8 +137,21 @@ public:
bool inserted; bool inserted;
this->emplace(x, it, inserted); this->emplace(x, it, inserted);
/// Если тривиальный конструктор, то инициализация нулями (через вызов конструктора для POD-ов) не нужна, так как таблица и так заполнена нулями. /** Может показаться, что инициализация не обязательна для POD-типов (или __has_trivial_constructor),
if (!__has_trivial_constructor(mapped_type) && inserted) * так как кусок памяти для хэш-таблицы изначально инициализирован нулями.
* Но, на самом деле, пустая ячейка может быть не инициализирована нулями в следующих случаях:
* - ZeroValueStorage (в нём зануляется только ключ);
* - после ресайза и переноса части ячеек в новую половину хэш-таблицы, у старых ячеек, тоже зануляется только ключ.
*
* По производительности, разницы почти всегда нет, за счёт того, что it->second как правило присваивается сразу
* после вызова operator[], и так как operator[] инлайнится, компилятор убирает лишнюю инициализацию.
*
* Иногда из-за инициализации, производительность даже растёт. Это происходит в коде вида ++map[key].
* Когда мы делаем инициализацию, то для новых ячеек, достаточно сразу сделать store 1.
* А если бы мы не делали инициализацию, то не смотря на то, что в ячейке был ноль,
* компилятор не может об этом догадаться, и генерирует код load, increment, store.
*/
if (inserted)
new(&it->second) mapped_type(); new(&it->second) mapped_type();
return it->second; return it->second;

View File

@ -185,6 +185,24 @@ public:
} }
/// То же самое, но вернуть false, если переполнено.
bool ALWAYS_INLINE tryEmplace(Key x, iterator & it, bool & inserted)
{
Cell * res = findCell(x);
it = iteratorTo(res);
inserted = res == buf + m_size;
if (inserted)
{
if (res == buf + capacity)
return false;
new(res) Cell(x, *this);
++m_size;
}
return true;
}
/// Скопировать ячейку из другой хэш-таблицы. Предполагается, что такого ключа в таблице ещё не было. /// Скопировать ячейку из другой хэш-таблицы. Предполагается, что такого ключа в таблице ещё не было.
void ALWAYS_INLINE insertUnique(const Cell * cell) void ALWAYS_INLINE insertUnique(const Cell * cell)
{ {
@ -192,6 +210,12 @@ public:
++m_size; ++m_size;
} }
void ALWAYS_INLINE insertUnique(Key x)
{
new(&buf[m_size]) Cell(x, *this);
++m_size;
}
iterator ALWAYS_INLINE find(Key x) { return iteratorTo(findCell(x)); } iterator ALWAYS_INLINE find(Key x) { return iteratorTo(findCell(x)); }
const_iterator ALWAYS_INLINE find(Key x) const { return iteratorTo(findCell(x)); } const_iterator ALWAYS_INLINE find(Key x) const { return iteratorTo(findCell(x)); }

View File

@ -27,7 +27,7 @@ public:
bool inserted; bool inserted;
this->emplace(x, it, inserted); this->emplace(x, it, inserted);
if (!__has_trivial_constructor(mapped_type) && inserted) if (inserted)
new(&it->second) mapped_type(); new(&it->second) mapped_type();
return it->second; return it->second;

View File

@ -10,13 +10,34 @@ namespace DB
struct BlockIO struct BlockIO
{ {
/** process_list_entry должен уничтожаться позже, чем in и out,
* так как внутри in и out есть ссылка на объект внутри process_list_entry
* (MemoryTracker * current_memory_tracker),
* которая может использоваться до уничтожения in и out.
*/
ProcessList::EntryPtr process_list_entry;
BlockInputStreamPtr in; BlockInputStreamPtr in;
BlockOutputStreamPtr out; BlockOutputStreamPtr out;
Block in_sample; /// Пример блока, который будет прочитан из in. Block in_sample; /// Пример блока, который будет прочитан из in.
Block out_sample; /// Пример блока, которого нужно писать в out. Block out_sample; /// Пример блока, которого нужно писать в out.
ProcessList::EntryPtr process_list_entry; BlockIO & operator= (const BlockIO & rhs)
{
/// Обеспечиваем правильный порядок уничтожения.
out = nullptr;
in = nullptr;
process_list_entry = nullptr;
process_list_entry = rhs.process_list_entry;
in = rhs.in;
out = rhs.out;
in_sample = rhs.in_sample;
out_sample = rhs.out_sample;
return *this;
}
}; };
} }

View File

@ -2,8 +2,6 @@
#include <DB/DataStreams/IProfilingBlockInputStream.h> #include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Interpreters/ExpressionAnalyzer.h> #include <DB/Interpreters/ExpressionAnalyzer.h>
#include <DB/Interpreters/Set.h>
#include <DB/Interpreters/Join.h>
namespace DB namespace DB

View File

@ -1,7 +1,7 @@
#pragma once #pragma once
#include <list> #include <list>
#include <queue> #include <stack>
#include <atomic> #include <atomic>
#include <thread> #include <thread>
#include <mutex> #include <mutex>
@ -49,7 +49,7 @@ public:
: inputs(inputs_), max_threads(std::min(inputs_.size(), max_threads_)), handler(handler_) : inputs(inputs_), max_threads(std::min(inputs_.size(), max_threads_)), handler(handler_)
{ {
for (size_t i = 0; i < inputs_.size(); ++i) for (size_t i = 0; i < inputs_.size(); ++i)
input_queue.emplace(inputs_[i], i); input_stack.emplace(inputs_[i], i);
} }
~ParallelInputsProcessor() ~ParallelInputsProcessor()
@ -162,16 +162,16 @@ private:
/// Выбираем следующий источник. /// Выбираем следующий источник.
{ {
std::lock_guard<std::mutex> lock(input_queue_mutex); std::lock_guard<std::mutex> lock(input_stack_mutex);
/// Если свободных источников нет, то этот поток больше не нужен. (Но другие потоки могут работать со своими источниками.) /// Если свободных источников нет, то этот поток больше не нужен. (Но другие потоки могут работать со своими источниками.)
if (input_queue.empty()) if (input_stack.empty())
break; break;
input = input_queue.front(); input = input_stack.top();
/// Убираем источник из очереди доступных источников. /// Убираем источник из очереди доступных источников.
input_queue.pop(); input_stack.pop();
} }
/// Основная работа. /// Основная работа.
@ -183,15 +183,15 @@ private:
/// Если этот источник ещё не иссяк, то положим полученный блок в очередь готовых. /// Если этот источник ещё не иссяк, то положим полученный блок в очередь готовых.
{ {
std::lock_guard<std::mutex> lock(input_queue_mutex); std::lock_guard<std::mutex> lock(input_stack_mutex);
if (block) if (block)
{ {
input_queue.push(input); input_stack.push(input);
} }
else else
{ {
if (input_queue.empty()) if (input_stack.empty())
break; break;
} }
} }
@ -214,12 +214,15 @@ private:
typedef std::vector<std::thread> ThreadsData; typedef std::vector<std::thread> ThreadsData;
ThreadsData threads; ThreadsData threads;
/// Очередь доступных источников, которые не заняты каким-либо потоком в данный момент. /** Стек доступных источников, которые не заняты каким-либо потоком в данный момент.
typedef std::queue<InputData> InputQueue; * Стек вместо очереди - чтобы выполнять работу по чтению одного источника более последовательно.
InputQueue input_queue; * То есть, продолжать обработку источника, который недавно обрабатывался.
*/
typedef std::stack<InputData> InputStack;
InputStack input_stack;
/// Для операций с input_queue. /// Для операций с input_stack.
std::mutex input_queue_mutex; std::mutex input_stack_mutex;
/// Сколько источников иссякло. /// Сколько источников иссякло.
std::atomic<size_t> active_threads { 0 }; std::atomic<size_t> active_threads { 0 };

View File

@ -22,13 +22,6 @@ class RemoteBlockInputStream : public IProfilingBlockInputStream
private: private:
void init(const Settings * settings_) void init(const Settings * settings_)
{ {
established.store(false, std::memory_order_seq_cst);
sent_query.store(false, std::memory_order_seq_cst);
finished.store(false, std::memory_order_seq_cst);
got_exception_from_replica.store(false, std::memory_order_seq_cst);
got_unknown_packet_from_replica.store(false, std::memory_order_seq_cst);
was_cancelled.store(false, std::memory_order_seq_cst);
if (settings_) if (settings_)
{ {
send_settings = true; send_settings = true;
@ -93,11 +86,7 @@ public:
if (hasNoQueryInProgress() || hasThrownException()) if (hasNoQueryInProgress() || hasThrownException())
return; return;
if (tryCancel()) tryCancel("Cancelling query");
{
std::string addresses = parallel_replicas->dumpAddresses();
LOG_TRACE(log, "(" + addresses + ") Cancelling query");
}
} }
@ -107,7 +96,7 @@ public:
* все соединения, затем читаем и пропускаем оставшиеся пакеты чтобы * все соединения, затем читаем и пропускаем оставшиеся пакеты чтобы
* эти соединения не остались висеть в рассихронизированном состоянии. * эти соединения не остались висеть в рассихронизированном состоянии.
*/ */
if (established.load(std::memory_order_seq_cst) || isQueryInProgress()) if (established || isQueryInProgress())
parallel_replicas->disconnect(); parallel_replicas->disconnect();
} }
@ -142,16 +131,16 @@ protected:
Block readImpl() override Block readImpl() override
{ {
if (!sent_query.load(std::memory_order_seq_cst)) if (!sent_query)
{ {
createParallelReplicas(); createParallelReplicas();
established.store(true, std::memory_order_seq_cst); established = true;
parallel_replicas->sendQuery(query, "", stage, true); parallel_replicas->sendQuery(query, "", stage, true);
established.store(false, std::memory_order_seq_cst); established = false;
sent_query.store(true, std::memory_order_seq_cst); sent_query = true;
sendExternalTables(); sendExternalTables();
} }
@ -169,14 +158,14 @@ protected:
break; /// Если блок пустой - получим другие пакеты до EndOfStream. break; /// Если блок пустой - получим другие пакеты до EndOfStream.
case Protocol::Server::Exception: case Protocol::Server::Exception:
got_exception_from_replica.store(true, std::memory_order_seq_cst); got_exception_from_replica = true;
packet.exception->rethrow(); packet.exception->rethrow();
break; break;
case Protocol::Server::EndOfStream: case Protocol::Server::EndOfStream:
if (!parallel_replicas->hasActiveReplicas()) if (!parallel_replicas->hasActiveReplicas())
{ {
finished.store(true, std::memory_order_seq_cst); finished = true;
return Block(); return Block();
} }
break; break;
@ -208,7 +197,7 @@ protected:
break; break;
default: default:
got_unknown_packet_from_replica.store(true, std::memory_order_seq_cst); got_unknown_packet_from_replica = true;
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
} }
} }
@ -231,27 +220,23 @@ protected:
*/ */
/// Отправим просьбу прервать выполнение запроса, если ещё не отправляли. /// Отправим просьбу прервать выполнение запроса, если ещё не отправляли.
if (tryCancel()) tryCancel("Cancelling query because enough data has been read");
{
std::string addresses = parallel_replicas->dumpAddresses();
LOG_TRACE(log, "(" + addresses + ") Cancelling query because enough data has been read");
}
/// Получим оставшиеся пакеты, чтобы не было рассинхронизации в соединениях с репликами. /// Получим оставшиеся пакеты, чтобы не было рассинхронизации в соединениях с репликами.
Connection::Packet packet = parallel_replicas->drain(); Connection::Packet packet = parallel_replicas->drain();
switch (packet.type) switch (packet.type)
{ {
case Protocol::Server::EndOfStream: case Protocol::Server::EndOfStream:
finished.store(true, std::memory_order_seq_cst); finished = true;
break; break;
case Protocol::Server::Exception: case Protocol::Server::Exception:
got_exception_from_replica.store(true, std::memory_order_seq_cst); got_exception_from_replica = true;
packet.exception->rethrow(); packet.exception->rethrow();
break; break;
default: default:
got_unknown_packet_from_replica.store(true, std::memory_order_seq_cst); got_unknown_packet_from_replica = true;
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
} }
} }
@ -269,19 +254,19 @@ protected:
/// Возвращает true, если запрос отправлен, а ещё не выполнен. /// Возвращает true, если запрос отправлен, а ещё не выполнен.
bool isQueryInProgress() const bool isQueryInProgress() const
{ {
return sent_query.load(std::memory_order_seq_cst) && !finished.load(std::memory_order_seq_cst) && !was_cancelled.load(std::memory_order_seq_cst); return sent_query && !finished && !was_cancelled;
} }
/// Возвращает true, если никакой запрос не отправлен или один запрос уже выполнен. /// Возвращает true, если никакой запрос не отправлен или один запрос уже выполнен.
bool hasNoQueryInProgress() const bool hasNoQueryInProgress() const
{ {
return !sent_query.load(std::memory_order_seq_cst) || finished.load(std::memory_order_seq_cst); return !sent_query || finished;
} }
/// Возвращает true, если исключение было выкинуто. /// Возвращает true, если исключение было выкинуто.
bool hasThrownException() const bool hasThrownException() const
{ {
return got_exception_from_replica.load(std::memory_order_seq_cst) || got_unknown_packet_from_replica.load(std::memory_order_seq_cst); return got_exception_from_replica || got_unknown_packet_from_replica;
} }
private: private:
@ -293,17 +278,14 @@ private:
} }
/// Отправить запрос на отмену всех соединений к репликам, если такой запрос ещё не был отправлен. /// Отправить запрос на отмену всех соединений к репликам, если такой запрос ещё не был отправлен.
bool tryCancel() void tryCancel(const char * reason)
{ {
bool old_val = false; bool old_val = false;
bool new_val = true; if (!was_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_seq_cst))
if (was_cancelled.compare_exchange_strong(old_val, new_val, std::memory_order_seq_cst, std::memory_order_relaxed)) return;
{
parallel_replicas->sendCancel(); LOG_TRACE(log, "(" << parallel_replicas->dumpAddresses() << ") " << reason);
return true; parallel_replicas->sendCancel();
}
else
return false;
} }
private: private:
@ -324,33 +306,33 @@ private:
Context context; Context context;
/// Установили соединения с репликами, но ещё не отправили запрос. /// Установили соединения с репликами, но ещё не отправили запрос.
std::atomic<bool> established; std::atomic<bool> established { false };
/// Отправили запрос (это делается перед получением первого блока). /// Отправили запрос (это делается перед получением первого блока).
std::atomic<bool> sent_query; std::atomic<bool> sent_query { false };
/** Получили все данные от всех реплик, до пакета EndOfStream. /** Получили все данные от всех реплик, до пакета EndOfStream.
* Если при уничтожении объекта, ещё не все данные считаны, * Если при уничтожении объекта, ещё не все данные считаны,
* то для того, чтобы не было рассинхронизации, на реплики отправляются просьбы прервать выполнение запроса, * то для того, чтобы не было рассинхронизации, на реплики отправляются просьбы прервать выполнение запроса,
* и после этого считываются все пакеты до EndOfStream. * и после этого считываются все пакеты до EndOfStream.
*/ */
std::atomic<bool> finished; std::atomic<bool> finished { false };
/** На каждую реплику была отправлена просьба прервать выполнение запроса, так как данные больше не нужны. /** На каждую реплику была отправлена просьба прервать выполнение запроса, так как данные больше не нужны.
* Это может быть из-за того, что данных достаточно (например, при использовании LIMIT), * Это может быть из-за того, что данных достаточно (например, при использовании LIMIT),
* или если на стороне клиента произошло исключение. * или если на стороне клиента произошло исключение.
*/ */
std::atomic<bool> was_cancelled; std::atomic<bool> was_cancelled { false };
/** С одной репилки было получено исключение. В этом случае получать больше пакетов или /** С одной репилки было получено исключение. В этом случае получать больше пакетов или
* просить прервать запрос на этой реплике не нужно. * просить прервать запрос на этой реплике не нужно.
*/ */
std::atomic<bool> got_exception_from_replica; std::atomic<bool> got_exception_from_replica { false };
/** С одной реплики был получен неизвестный пакет. В этом случае получать больше пакетов или /** С одной реплики был получен неизвестный пакет. В этом случае получать больше пакетов или
* просить прервать запрос на этой реплике не нужно. * просить прервать запрос на этой реплике не нужно.
*/ */
std::atomic<bool> got_unknown_packet_from_replica; std::atomic<bool> got_unknown_packet_from_replica { false };
Logger * log = &Logger::get("RemoteBlockInputStream"); Logger * log = &Logger::get("RemoteBlockInputStream");
}; };

View File

@ -5,6 +5,7 @@
#include <DB/Dictionaries/DictionaryStructure.h> #include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Common/HashTable/HashMap.h> #include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>
#include <DB/Common/HashTable/HashMap.h>
#include <statdaemons/ext/scope_guard.hpp> #include <statdaemons/ext/scope_guard.hpp>
#include <Poco/RWLock.h> #include <Poco/RWLock.h>
#include <cmath> #include <cmath>
@ -413,6 +414,15 @@ private:
auto stream = source_ptr->loadIds(ids); auto stream = source_ptr->loadIds(ids);
stream->readPrefix(); stream->readPrefix();
HashMap<UInt64, UInt8> remaining_ids{ids.size()};
for (const auto id : ids)
remaining_ids.insert({ id, 0 });
std::uniform_int_distribution<std::uint64_t> distribution{
dict_lifetime.min_sec,
dict_lifetime.max_sec
};
const Poco::ScopedWriteRWLock write_lock{rw_lock}; const Poco::ScopedWriteRWLock write_lock{rw_lock};
while (const auto block = stream->read()) while (const auto block = stream->read())
@ -434,7 +444,7 @@ private:
for (const auto i : ext::range(0, ids.size())) for (const auto i : ext::range(0, ids.size()))
{ {
const auto id = ids[i]; const auto id = ids[i];
const auto & cell_idx = getCellIdx(id); const auto cell_idx = getCellIdx(id);
auto & cell = cells[cell_idx]; auto & cell = cells[cell_idx];
for (const auto attribute_idx : ext::range(0, attributes.size())) for (const auto attribute_idx : ext::range(0, attributes.size()))
@ -445,19 +455,33 @@ private:
setAttributeValue(attribute, cell_idx, attribute_column[i]); setAttributeValue(attribute, cell_idx, attribute_column[i]);
} }
std::uniform_int_distribution<std::uint64_t> distribution{
dict_lifetime.min_sec,
dict_lifetime.max_sec
};
cell.id = id; cell.id = id;
cell.expires_at = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}; cell.expires_at = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
on_cell_updated(id, cell_idx); on_cell_updated(id, cell_idx);
remaining_ids[id] = 1;
} }
} }
stream->readSuffix(); stream->readSuffix();
for (const auto id_found_pair : remaining_ids)
{
if (id_found_pair.second)
continue;
const auto id = id_found_pair.first;
const auto cell_idx = getCellIdx(id);
auto & cell = cells[cell_idx];
for (auto & attribute : attributes)
setDefaultAttributeValue(attribute, cell_idx);
cell.id = id;
cell.expires_at = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
on_cell_updated(id, cell_idx);
}
} }
std::uint64_t getCellIdx(const id_t id) const std::uint64_t getCellIdx(const id_t id) const
@ -467,6 +491,36 @@ private:
return idx; return idx;
} }
void setDefaultAttributeValue(attribute_t & attribute, const id_t idx) const
{
switch (attribute.type)
{
case AttributeType::uint8: std::get<std::unique_ptr<UInt8[]>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values); break;
case AttributeType::uint16: std::get<std::unique_ptr<UInt16[]>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values); break;
case AttributeType::uint32: std::get<std::unique_ptr<UInt32[]>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values); break;
case AttributeType::uint64: std::get<std::unique_ptr<UInt64[]>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values); break;
case AttributeType::int8: std::get<std::unique_ptr<Int8[]>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values); break;
case AttributeType::int16: std::get<std::unique_ptr<Int16[]>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values); break;
case AttributeType::int32: std::get<std::unique_ptr<Int32[]>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values); break;
case AttributeType::int64: std::get<std::unique_ptr<Int64[]>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values); break;
case AttributeType::float32: std::get<std::unique_ptr<Float32[]>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values); break;
case AttributeType::float64: std::get<std::unique_ptr<Float64[]>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values); break;
case AttributeType::string:
{
const auto & null_value_ref = std::get<String>(attribute.null_values);
auto & string_ref = std::get<std::unique_ptr<StringRef[]>>(attribute.arrays)[idx];
if (string_ref.data == null_value_ref.data())
return;
delete[] string_ref.data;
string_ref = StringRef{null_value_ref};
break;
}
}
}
void setAttributeValue(attribute_t & attribute, const id_t idx, const Field & value) const void setAttributeValue(attribute_t & attribute, const id_t idx, const Field & value) const
{ {
switch (attribute.type) switch (attribute.type)
@ -485,7 +539,8 @@ private:
{ {
const auto & string = value.get<String>(); const auto & string = value.get<String>();
auto & string_ref = std::get<std::unique_ptr<StringRef[]>>(attribute.arrays)[idx]; auto & string_ref = std::get<std::unique_ptr<StringRef[]>>(attribute.arrays)[idx];
if (string_ref.data) const auto & null_value_ref = std::get<String>(attribute.null_values);
if (string_ref.data != null_value_ref.data())
delete[] string_ref.data; delete[] string_ref.data;
const auto size = string.size(); const auto size = string.size();

View File

@ -11,7 +11,7 @@
namespace DB namespace DB
{ {
const auto max_connections = 1; const auto max_connections = 16;
/** Allows loading dictionaries from local or remote ClickHouse instance /** Allows loading dictionaries from local or remote ClickHouse instance
* @todo use ConnectionPoolWithFailover * @todo use ConnectionPoolWithFailover
@ -75,56 +75,70 @@ public:
DictionarySourcePtr clone() const override { return std::make_unique<ClickHouseDictionarySource>(*this); } DictionarySourcePtr clone() const override { return std::make_unique<ClickHouseDictionarySource>(*this); }
private: private:
/// @todo escape table and column names
static std::string composeLoadAllQuery(const Block & block, const std::string & table) static std::string composeLoadAllQuery(const Block & block, const std::string & table)
{ {
std::string query{"SELECT "}; std::string query;
auto first = true;
for (const auto idx : ext::range(0, block.columns()))
{ {
if (!first) WriteBufferFromString out{query};
query += ", "; writeString("SELECT ", out);
query += block.getByPosition(idx).name; auto first = true;
first = false; for (const auto idx : ext::range(0, block.columns()))
{
if (!first)
writeString(", ", out);
writeString(block.getByPosition(idx).name, out);
first = false;
}
writeString(" FROM ", out);
writeProbablyBackQuotedString(table, out);
writeChar(';', out);
} }
query += " FROM " + table + ';';
return query; return query;
} }
std::string composeLoadIdsQuery(const std::vector<std::uint64_t> ids) std::string composeLoadIdsQuery(const std::vector<std::uint64_t> ids)
{ {
std::string query{"SELECT "}; std::string query;
auto first = true;
for (const auto idx : ext::range(0, sample_block.columns()))
{ {
if (!first) WriteBufferFromString out{query};
query += ", "; writeString("SELECT ", out);
first = false; auto first = true;
query += sample_block.getByPosition(idx).name; for (const auto idx : ext::range(0, sample_block.columns()))
{
if (!first)
writeString(", ", out);
writeString(sample_block.getByPosition(idx).name, out);
first = false;
}
const auto & id_column_name = sample_block.getByPosition(0).name;
writeString(" FROM ", out);
writeProbablyBackQuotedString(table, out);
writeString(" WHERE ", out);
writeProbablyBackQuotedString(id_column_name, out);
writeString(" IN (", out);
first = true;
for (const auto id : ids)
{
if (!first)
writeString(", ", out);
first = false;
writeString(toString(id), out);
}
writeString(");", out);
} }
const auto & id_column_name = sample_block.getByPosition(0).name;
query += " FROM " + table + " WHERE " + id_column_name + " IN (";
first = true;
for (const auto id : ids)
{
if (!first)
query += ',';
first = false;
query += toString(id);
}
query += ");";
return query; return query;
} }

View File

@ -165,6 +165,7 @@ private:
{ {
const auto size = dict_struct.attributes.size(); const auto size = dict_struct.attributes.size();
attributes.reserve(size); attributes.reserve(size);
for (const auto & attribute : dict_struct.attributes) for (const auto & attribute : dict_struct.attributes)
{ {
attribute_index_by_name.emplace(attribute.name, attributes.size()); attribute_index_by_name.emplace(attribute.name, attributes.size());
@ -275,56 +276,16 @@ private:
switch (attribute.type) switch (attribute.type)
{ {
case AttributeType::uint8: case AttributeType::uint8: setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>()); break;
{ case AttributeType::uint16: setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>()); break;
setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>()); case AttributeType::uint32: setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>()); break;
break; case AttributeType::uint64: setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>()); break;
} case AttributeType::int8: setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>()); break;
case AttributeType::uint16: case AttributeType::int16: setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>()); break;
{ case AttributeType::int32: setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>()); break;
setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>()); case AttributeType::int64: setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>()); break;
break; case AttributeType::float32: setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>()); break;
} case AttributeType::float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeType::uint32:
{
setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>());
break;
}
case AttributeType::uint64:
{
setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>());
break;
}
case AttributeType::int8:
{
setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>());
break;
}
case AttributeType::int16:
{
setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>());
break;
}
case AttributeType::int32:
{
setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>());
break;
}
case AttributeType::int64:
{
setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>());
break;
}
case AttributeType::float32:
{
setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>());
break;
}
case AttributeType::float64:
{
setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>());
break;
}
case AttributeType::string: case AttributeType::string:
{ {
auto & array = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays); auto & array = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays);

View File

@ -7,6 +7,7 @@
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp> #include <statdaemons/ext/range.hpp>
#include <memory> #include <memory>
#include <tuple>
namespace DB namespace DB
{ {
@ -44,21 +45,15 @@ public:
id_t toParent(const id_t id) const override id_t toParent(const id_t id) const override
{ {
const auto attr = hierarchical_attribute; const auto attr = hierarchical_attribute;
const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, UInt64>>>(attr->maps);
const auto it = map.find(id);
const auto it = attr->uint64_map->find(id); return it != map.end() ? it->second : std::get<UInt64>(attr->null_values);
return it != attr->uint64_map->end() ? it->second : attr->uint64_null_value;
} }
void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const override void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const override
{ {
const auto & attr = *hierarchical_attribute->uint64_map; getItems<UInt64>(*hierarchical_attribute, ids, out);
const auto null_value = hierarchical_attribute->uint64_null_value;
for (const auto i : ext::range(0, ids.size()))
{
const auto it = attr.find(ids[i]);
out[i] = it != attr.end() ? it->second : null_value;
}
} }
#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \ #define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \
@ -71,11 +66,10 @@ public:
ErrorCodes::TYPE_MISMATCH\ ErrorCodes::TYPE_MISMATCH\
};\ };\
\ \
const auto it = attribute.LC_TYPE##_map->find(id);\ const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, TYPE>>>(attribute.maps);\
if (it != attribute.LC_TYPE##_map->end())\ const auto it = map.find(id);\
return TYPE{it->second};\
\ \
return attribute.LC_TYPE##_null_value;\ return it != map.end() ? TYPE{it->second} : std::get<TYPE>(attribute.null_values);\
} }
DECLARE_INDIVIDUAL_GETTER(UInt8, uint8) DECLARE_INDIVIDUAL_GETTER(UInt8, uint8)
DECLARE_INDIVIDUAL_GETTER(UInt16, uint16) DECLARE_INDIVIDUAL_GETTER(UInt16, uint16)
@ -87,8 +81,21 @@ public:
DECLARE_INDIVIDUAL_GETTER(Int64, int64) DECLARE_INDIVIDUAL_GETTER(Int64, int64)
DECLARE_INDIVIDUAL_GETTER(Float32, float32) DECLARE_INDIVIDUAL_GETTER(Float32, float32)
DECLARE_INDIVIDUAL_GETTER(Float64, float64) DECLARE_INDIVIDUAL_GETTER(Float64, float64)
DECLARE_INDIVIDUAL_GETTER(String, string)
#undef DECLARE_INDIVIDUAL_GETTER #undef DECLARE_INDIVIDUAL_GETTER
String getString(const std::string & attribute_name, const id_t id) const override
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
const auto it = map.find(id);
return it != map.end() ? String{it->second} : std::get<String>(attribute.null_values);
}
#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\ #define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\ void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
@ -100,14 +107,7 @@ public:
ErrorCodes::TYPE_MISMATCH\ ErrorCodes::TYPE_MISMATCH\
};\ };\
\ \
const auto & attr = *attribute.LC_TYPE##_map;\ getItems<TYPE>(attribute, ids, out);\
const auto null_value = attribute.LC_TYPE##_null_value;\
\
for (const auto i : ext::range(0, ids.size()))\
{\
const auto it = attr.find(ids[i]);\
out[i] = it != attr.end() ? it->second : null_value;\
}\
} }
DECLARE_MULTIPLE_GETTER(UInt8, uint8) DECLARE_MULTIPLE_GETTER(UInt8, uint8)
DECLARE_MULTIPLE_GETTER(UInt16, uint16) DECLARE_MULTIPLE_GETTER(UInt16, uint16)
@ -129,8 +129,8 @@ public:
ErrorCodes::TYPE_MISMATCH ErrorCodes::TYPE_MISMATCH
}; };
const auto & attr = *attribute.string_map; const auto & attr = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
const auto null_value = attribute.string_null_value; const auto & null_value = std::get<String>(attribute.null_values);
for (const auto i : ext::range(0, ids.size())) for (const auto i : ext::range(0, ids.size()))
{ {
@ -141,38 +141,32 @@ public:
} }
private: private:
struct attribute_t struct attribute_t final
{ {
AttributeType type; AttributeType type;
UInt8 uint8_null_value; std::tuple<UInt8, UInt16, UInt32, UInt64,
UInt16 uint16_null_value; Int8, Int16, Int32, Int64,
UInt32 uint32_null_value; Float32, Float64,
UInt64 uint64_null_value; String> null_values;
Int8 int8_null_value; std::tuple<std::unique_ptr<HashMap<UInt64, UInt8>>,
Int16 int16_null_value; std::unique_ptr<HashMap<UInt64, UInt16>>,
Int32 int32_null_value; std::unique_ptr<HashMap<UInt64, UInt32>>,
Int64 int64_null_value; std::unique_ptr<HashMap<UInt64, UInt64>>,
Float32 float32_null_value; std::unique_ptr<HashMap<UInt64, Int8>>,
Float64 float64_null_value; std::unique_ptr<HashMap<UInt64, Int16>>,
String string_null_value; std::unique_ptr<HashMap<UInt64, Int32>>,
std::unique_ptr<HashMap<UInt64, UInt8>> uint8_map; std::unique_ptr<HashMap<UInt64, Int64>>,
std::unique_ptr<HashMap<UInt64, UInt16>> uint16_map; std::unique_ptr<HashMap<UInt64, Float32>>,
std::unique_ptr<HashMap<UInt64, UInt32>> uint32_map; std::unique_ptr<HashMap<UInt64, Float64>>,
std::unique_ptr<HashMap<UInt64, UInt64>> uint64_map; std::unique_ptr<HashMap<UInt64, StringRef>>> maps;
std::unique_ptr<HashMap<UInt64, Int8>> int8_map;
std::unique_ptr<HashMap<UInt64, Int16>> int16_map;
std::unique_ptr<HashMap<UInt64, Int32>> int32_map;
std::unique_ptr<HashMap<UInt64, Int64>> int64_map;
std::unique_ptr<HashMap<UInt64, Float32>> float32_map;
std::unique_ptr<HashMap<UInt64, Float64>> float64_map;
std::unique_ptr<Arena> string_arena; std::unique_ptr<Arena> string_arena;
std::unique_ptr<HashMap<UInt64, StringRef>> string_map;
}; };
void createAttributes() void createAttributes()
{ {
const auto size = dict_struct.attributes.size(); const auto size = dict_struct.attributes.size();
attributes.reserve(size); attributes.reserve(size);
for (const auto & attribute : dict_struct.attributes) for (const auto & attribute : dict_struct.attributes)
{ {
attribute_index_by_name.emplace(attribute.name, attributes.size()); attribute_index_by_name.emplace(attribute.name, attributes.size());
@ -214,124 +208,85 @@ private:
stream->readSuffix(); stream->readSuffix();
} }
template <typename T>
void createAttributeImpl(attribute_t & attribute, const std::string & null_value)
{
std::get<T>(attribute.null_values) = DB::parse<T>(null_value);
std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps) = std::make_unique<HashMap<UInt64, T>>();
}
attribute_t createAttributeWithType(const AttributeType type, const std::string & null_value) attribute_t createAttributeWithType(const AttributeType type, const std::string & null_value)
{ {
attribute_t attr{type}; attribute_t attr{type};
switch (type) switch (type)
{ {
case AttributeType::uint8: case AttributeType::uint8: createAttributeImpl<UInt8>(attr, null_value); break;
attr.uint8_null_value = DB::parse<UInt8>(null_value); case AttributeType::uint16: createAttributeImpl<UInt16>(attr, null_value); break;
attr.uint8_map.reset(new HashMap<UInt64, UInt8>); case AttributeType::uint32: createAttributeImpl<UInt32>(attr, null_value); break;
break; case AttributeType::uint64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeType::uint16: case AttributeType::int8: createAttributeImpl<Int8>(attr, null_value); break;
attr.uint16_null_value = DB::parse<UInt16>(null_value); case AttributeType::int16: createAttributeImpl<Int16>(attr, null_value); break;
attr.uint16_map.reset(new HashMap<UInt64, UInt16>); case AttributeType::int32: createAttributeImpl<Int32>(attr, null_value); break;
break; case AttributeType::int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeType::uint32: case AttributeType::float32: createAttributeImpl<Float32>(attr, null_value); break;
attr.uint32_null_value = DB::parse<UInt32>(null_value); case AttributeType::float64: createAttributeImpl<Float64>(attr, null_value); break;
attr.uint32_map.reset(new HashMap<UInt64, UInt32>);
break;
case AttributeType::uint64:
attr.uint64_null_value = DB::parse<UInt64>(null_value);
attr.uint64_map.reset(new HashMap<UInt64, UInt64>);
break;
case AttributeType::int8:
attr.int8_null_value = DB::parse<Int8>(null_value);
attr.int8_map.reset(new HashMap<UInt64, Int8>);
break;
case AttributeType::int16:
attr.int16_null_value = DB::parse<Int16>(null_value);
attr.int16_map.reset(new HashMap<UInt64, Int16>);
break;
case AttributeType::int32:
attr.int32_null_value = DB::parse<Int32>(null_value);
attr.int32_map.reset(new HashMap<UInt64, Int32>);
break;
case AttributeType::int64:
attr.int64_null_value = DB::parse<Int64>(null_value);
attr.int64_map.reset(new HashMap<UInt64, Int64>);
break;
case AttributeType::float32:
attr.float32_null_value = DB::parse<Float32>(null_value);
attr.float32_map.reset(new HashMap<UInt64, Float32>);
break;
case AttributeType::float64:
attr.float64_null_value = DB::parse<Float64>(null_value);
attr.float64_map.reset(new HashMap<UInt64, Float64>);
break;
case AttributeType::string: case AttributeType::string:
attr.string_null_value = null_value; {
attr.string_arena.reset(new Arena); const auto & null_value_ref = std::get<String>(attr.null_values) = DB::parse<String>(null_value);
attr.string_map.reset(new HashMap<UInt64, StringRef>); std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attr.maps) =
std::make_unique<HashMap<UInt64, StringRef>>();
attr.string_arena = std::make_unique<Arena>();
break; break;
}
} }
return attr; return attr;
} }
template <typename T>
void getItems(const attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out) const
{
const auto & attr = *std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps);
const auto null_value = std::get<T>(attribute.null_values);
for (const auto i : ext::range(0, ids.size()))
{
const auto it = attr.find(ids[i]);
out[i] = it != attr.end() ? it->second : null_value;
}
}
template <typename T>
void setAttributeValueImpl(attribute_t & attribute, const id_t id, const T value)
{
auto & map = *std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps);
map.insert({ id, value });
}
void setAttributeValue(attribute_t & attribute, const id_t id, const Field & value) void setAttributeValue(attribute_t & attribute, const id_t id, const Field & value)
{ {
switch (attribute.type) switch (attribute.type)
{ {
case AttributeType::uint8: case AttributeType::uint8: setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>()); break;
{ case AttributeType::uint16: setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>()); break;
attribute.uint8_map->insert({ id, value.get<UInt64>() }); case AttributeType::uint32: setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>()); break;
break; case AttributeType::uint64: setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>()); break;
} case AttributeType::int8: setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>()); break;
case AttributeType::uint16: case AttributeType::int16: setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>()); break;
{ case AttributeType::int32: setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>()); break;
attribute.uint16_map->insert({ id, value.get<UInt64>() }); case AttributeType::int64: setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>()); break;
break; case AttributeType::float32: setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>()); break;
} case AttributeType::float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeType::uint32:
{
attribute.uint32_map->insert({ id, value.get<UInt64>() });
break;
}
case AttributeType::uint64:
{
attribute.uint64_map->insert({ id, value.get<UInt64>() });
break;
}
case AttributeType::int8:
{
attribute.int8_map->insert({ id, value.get<Int64>() });
break;
}
case AttributeType::int16:
{
attribute.int16_map->insert({ id, value.get<Int64>() });
break;
}
case AttributeType::int32:
{
attribute.int32_map->insert({ id, value.get<Int64>() });
break;
}
case AttributeType::int64:
{
attribute.int64_map->insert({ id, value.get<Int64>() });
break;
}
case AttributeType::float32:
{
attribute.float32_map->insert({ id, value.get<Float64>() });
break;
}
case AttributeType::float64:
{
attribute.float64_map->insert({ id, value.get<Float64>() });
break;
}
case AttributeType::string: case AttributeType::string:
{ {
auto & map = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
const auto & string = value.get<String>(); const auto & string = value.get<String>();
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size()); const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
attribute.string_map->insert({ id, StringRef{string_in_arena, string.size()} }); map.insert({ id, StringRef{string_in_arena, string.size()} });
break; break;
} }
}; }
} }
const attribute_t & getAttribute(const std::string & attribute_name) const const attribute_t & getAttribute(const std::string & attribute_name) const

View File

@ -41,10 +41,10 @@ public:
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> ids) override BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> ids) override
{ {
throw Exception{ last_modification = getLastModification();
"Method unsupported", const auto query = composeLoadIdsQuery(ids);
ErrorCodes::NOT_IMPLEMENTED
}; return new MySQLBlockInputStream{pool.Get()->query(query), sample_block, max_block_size};
} }
bool isModified() const override { return getLastModification() > last_modification; } bool isModified() const override { return getLastModification() > last_modification; }
@ -76,22 +76,69 @@ private:
return mysqlxx::DateTime{std::time(nullptr)}; return mysqlxx::DateTime{std::time(nullptr)};
} }
/// @todo escape table and column names
static std::string composeLoadAllQuery(const Block & block, const std::string & table) static std::string composeLoadAllQuery(const Block & block, const std::string & table)
{ {
std::string query{"SELECT "}; std::string query;
auto first = true;
for (const auto idx : ext::range(0, block.columns()))
{ {
if (!first) WriteBufferFromString out{query};
query += ", "; writeString("SELECT ", out);
query += block.getByPosition(idx).name; auto first = true;
first = false; for (const auto idx : ext::range(0, block.columns()))
{
if (!first)
writeString(", ", out);
writeString(block.getByPosition(idx).name, out);
first = false;
}
writeString(" FROM ", out);
writeProbablyBackQuotedString(table, out);
writeChar(';', out);
} }
query += " FROM " + table + ';'; return query;
}
std::string composeLoadIdsQuery(const std::vector<std::uint64_t> ids)
{
std::string query;
{
WriteBufferFromString out{query};
writeString("SELECT ", out);
auto first = true;
for (const auto idx : ext::range(0, sample_block.columns()))
{
if (!first)
writeString(", ", out);
writeString(sample_block.getByPosition(idx).name, out);
first = false;
}
const auto & id_column_name = sample_block.getByPosition(0).name;
writeString(" FROM ", out);
writeProbablyBackQuotedString(table, out);
writeString(" WHERE ", out);
writeProbablyBackQuotedString(id_column_name, out);
writeString(" IN (", out);
first = true;
for (const auto id : ids)
{
if (!first)
writeString(", ", out);
first = false;
writeString(toString(id), out);
}
writeString(");", out);
}
return query; return query;
} }

View File

@ -1324,7 +1324,6 @@ public:
if (col_fstr_in) if (col_fstr_in)
{ {
ColumnString * col_str = new ColumnString; ColumnString * col_str = new ColumnString;
col_res = col_str; col_res = col_str;
ColumnString::Chars_t & out_vec = col_str->getChars(); ColumnString::Chars_t & out_vec = col_str->getChars();

View File

@ -408,6 +408,7 @@ public:
if (const ColumnUInt32 * times = typeid_cast<const ColumnUInt32 *>(&*block.getByPosition(arguments[0]).column)) if (const ColumnUInt32 * times = typeid_cast<const ColumnUInt32 *>(&*block.getByPosition(arguments[0]).column))
{ {
ColumnUInt32 * res = new ColumnUInt32; ColumnUInt32 * res = new ColumnUInt32;
ColumnPtr res_holder = res;
ColumnUInt32::Container_t & res_vec = res->getData(); ColumnUInt32::Container_t & res_vec = res->getData();
const ColumnUInt32::Container_t & vec = times->getData(); const ColumnUInt32::Container_t & vec = times->getData();
@ -417,7 +418,7 @@ public:
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
res_vec[i] = vec[i] / TIME_SLOT_SIZE * TIME_SLOT_SIZE; res_vec[i] = vec[i] / TIME_SLOT_SIZE * TIME_SLOT_SIZE;
block.getByPosition(result).column = res; block.getByPosition(result).column = res_holder;
} }
else if (const ColumnConstUInt32 * const_times = typeid_cast<const ColumnConstUInt32 *>(&*block.getByPosition(arguments[0]).column)) else if (const ColumnConstUInt32 * const_times = typeid_cast<const ColumnConstUInt32 *>(&*block.getByPosition(arguments[0]).column))
{ {
@ -551,22 +552,23 @@ public:
const ColumnConstUInt32 * const_durations = typeid_cast<const ColumnConstUInt32 *>(&*block.getByPosition(arguments[1]).column); const ColumnConstUInt32 * const_durations = typeid_cast<const ColumnConstUInt32 *>(&*block.getByPosition(arguments[1]).column);
ColumnArray * res = new ColumnArray(new ColumnUInt32); ColumnArray * res = new ColumnArray(new ColumnUInt32);
ColumnPtr res_holder = res;
ColumnUInt32::Container_t & res_values = typeid_cast<ColumnUInt32 &>(res->getData()).getData(); ColumnUInt32::Container_t & res_values = typeid_cast<ColumnUInt32 &>(res->getData()).getData();
if (starts && durations) if (starts && durations)
{ {
TimeSlotsImpl<UInt32>::vector_vector(starts->getData(), durations->getData(), res_values, res->getOffsets()); TimeSlotsImpl<UInt32>::vector_vector(starts->getData(), durations->getData(), res_values, res->getOffsets());
block.getByPosition(result).column = res; block.getByPosition(result).column = res_holder;
} }
else if (starts && const_durations) else if (starts && const_durations)
{ {
TimeSlotsImpl<UInt32>::vector_constant(starts->getData(), const_durations->getData(), res_values, res->getOffsets()); TimeSlotsImpl<UInt32>::vector_constant(starts->getData(), const_durations->getData(), res_values, res->getOffsets());
block.getByPosition(result).column = res; block.getByPosition(result).column = res_holder;
} }
else if (const_starts && durations) else if (const_starts && durations)
{ {
TimeSlotsImpl<UInt32>::constant_vector(const_starts->getData(), durations->getData(), res_values, res->getOffsets()); TimeSlotsImpl<UInt32>::constant_vector(const_starts->getData(), durations->getData(), res_values, res->getOffsets());
block.getByPosition(result).column = res; block.getByPosition(result).column = res_holder;
} }
else if (const_starts && const_durations) else if (const_starts && const_durations)
{ {

View File

@ -1107,26 +1107,61 @@ private:
const auto id_col_untyped = block.getByPosition(arguments[1]).column.get(); const auto id_col_untyped = block.getByPosition(arguments[1]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped)) if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
{ {
const auto & in = id_col->getData();
const auto size = in.size();
/// copy of `in` array
auto in_array = std::make_unique<PODArray<UInt64>>(std::begin(in), std::end(in));
/// used for storing and handling result of ::toParent call
auto out_array = std::make_unique<PODArray<UInt64>>(size);
/// resulting hierarchies
std::vector<std::vector<IDictionary::id_t>> hierarchies(size);
/// total number of non-zero elements, used for allocating all the required memory upfront
std::size_t total_count = 0;
while (true)
{
auto all_zeroes = true;
/// erase zeroed identifiers, store non-zeroed ones
for (const auto i : ext::range(0, size))
{
const auto id = (*in_array)[i];
if (0 == id)
continue;
all_zeroes = false;
/// place id at it's corresponding place
hierarchies[i].push_back(id);
++total_count;
}
if (all_zeroes)
break;
/// translate all non-zero identifiers at once
dictionary->toParent(*in_array, *out_array);
/// we're going to use the `in_array` from this iteration as `out_array` on the next one
std::swap(in_array, out_array);
}
const auto backend = new ColumnVector<UInt64>; const auto backend = new ColumnVector<UInt64>;
const auto array = new ColumnArray{backend}; const auto array = new ColumnArray{backend};
block.getByPosition(result).column = array; block.getByPosition(result).column = array;
const auto & in = id_col->getData();
const auto size = in.size();
auto & out = backend->getData(); auto & out = backend->getData();
auto & offsets = array->getOffsets(); auto & offsets = array->getOffsets();
out.reserve(total_count);
offsets.resize(size); offsets.resize(size);
out.reserve(size * 4);
for (const auto idx : ext::range(0, size)) for (const auto i : ext::range(0, size))
{ {
IDictionary::id_t cur = in[idx]; const auto & ids = hierarchies[i];
while (cur) out.insert_assume_reserved(std::begin(ids), std::end(ids));
{ offsets[i] = out.size();
out.push_back(cur);
cur = dictionary->toParent(cur);
}
offsets[idx] = out.size();
} }
} }
else if (const auto id_col = typeid_cast<const ColumnConst<UInt64> *>(id_col_untyped)) else if (const auto id_col = typeid_cast<const ColumnConst<UInt64> *>(id_col_untyped))

View File

@ -1408,23 +1408,6 @@ private:
ErrorCodes::ILLEGAL_COLUMN ErrorCodes::ILLEGAL_COLUMN
}; };
} }
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,
ColumnString::Chars_t & res_data, ColumnString::Offsets_t & res_offsets)
{
res_data.resize(data.size());
res_offsets.assign(offsets);
size_t size = offsets.size();
ColumnString::Offset_t prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
for (size_t j = prev_offset; j < offsets[i] - 1; ++j)
res_data[j] = data[offsets[i] + prev_offset - 2 - j];
res_data[offsets[i] - 1] = 0;
prev_offset = offsets[i];
}
}
}; };

View File

@ -338,6 +338,7 @@ public:
typeid_cast<const ColumnConstString *>(&*block.getByPosition(arrayArgumentPosition).column); typeid_cast<const ColumnConstString *>(&*block.getByPosition(arrayArgumentPosition).column);
ColumnArray * col_res = new ColumnArray(new ColumnString); ColumnArray * col_res = new ColumnArray(new ColumnString);
ColumnPtr col_res_holder = col_res;
ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData()); ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData());
ColumnArray::Offsets_t & res_offsets = col_res->getOffsets(); ColumnArray::Offsets_t & res_offsets = col_res->getOffsets();
ColumnString::Chars_t & res_strings_chars = res_strings.getChars(); ColumnString::Chars_t & res_strings_chars = res_strings.getChars();
@ -385,7 +386,7 @@ public:
res_offsets.push_back(current_dst_offset); res_offsets.push_back(current_dst_offset);
} }
block.getByPosition(result).column = col_res; block.getByPosition(result).column = col_res_holder;
} }
else if (col_const_str) else if (col_const_str)
{ {

View File

@ -113,6 +113,7 @@ private:
return; return;
free(reinterpret_cast<void *>(m_data)); free(reinterpret_cast<void *>(m_data));
m_data = nullptr; /// Чтобы избежать double free, если последующий вызов alloc кинет исключение.
if (current_memory_tracker) if (current_memory_tracker)
current_memory_tracker->free(m_capacity); current_memory_tracker->free(m_capacity);

View File

@ -40,6 +40,8 @@ private:
mutable std::mutex dictionaries_mutex; mutable std::mutex dictionaries_mutex;
std::unordered_map<std::string, std::shared_ptr<MultiVersion<IDictionary>>> dictionaries; std::unordered_map<std::string, std::shared_ptr<MultiVersion<IDictionary>>> dictionaries;
/// exception pointers for notifying user about failures on dictionary creation
std::unordered_map<std::string, std::exception_ptr> stored_exceptions;
std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times; std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
std::mt19937_64 rnd_engine{getSeed()}; std::mt19937_64 rnd_engine{getSeed()};
@ -52,24 +54,6 @@ private:
Poco::Timestamp config_last_modified{0}; Poco::Timestamp config_last_modified{0};
void handleException() const
{
try
{
throw;
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "Cannot load exter dictionary! You must resolve this manually. " << e.displayText());
return;
}
catch (...)
{
LOG_ERROR(log, "Cannot load dictionary! You must resolve this manually.");
return;
}
}
void reloadImpl(); void reloadImpl();
void reloadPeriodically() void reloadPeriodically()
@ -110,10 +94,16 @@ public:
const std::lock_guard<std::mutex> lock{dictionaries_mutex}; const std::lock_guard<std::mutex> lock{dictionaries_mutex};
const auto it = dictionaries.find(name); const auto it = dictionaries.find(name);
if (it == std::end(dictionaries)) if (it == std::end(dictionaries))
throw Exception{ {
"No such dictionary: " + name, const auto exception_it = stored_exceptions.find(name);
ErrorCodes::BAD_ARGUMENTS if (exception_it != std::end(stored_exceptions))
}; std::rethrow_exception(exception_it->second);
else
throw Exception{
"No such dictionary: " + name,
ErrorCodes::BAD_ARGUMENTS
};
}
return it->second->get(); return it->second->get();
} }

View File

@ -5,7 +5,6 @@
#include <DB/Parsers/ASTJoin.h> #include <DB/Parsers/ASTJoin.h>
#include <DB/Interpreters/AggregationCommon.h> #include <DB/Interpreters/AggregationCommon.h>
#include <DB/Interpreters/Set.h>
#include <DB/Common/Arena.h> #include <DB/Common/Arena.h>
#include <DB/Common/HashTable/HashMap.h> #include <DB/Common/HashTable/HashMap.h>
@ -67,7 +66,7 @@ public:
{ {
} }
bool empty() { return type == Set::EMPTY; } bool empty() { return type == Type::EMPTY; }
/** Добавить в отображение для соединения блок "правой" таблицы. /** Добавить в отображение для соединения блок "правой" таблицы.
* Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять. * Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
@ -155,7 +154,17 @@ private:
/// Дополнительные данные - строки, а также продолжения односвязных списков строк. /// Дополнительные данные - строки, а также продолжения односвязных списков строк.
Arena pool; Arena pool;
Set::Type type = Set::EMPTY; enum class Type
{
EMPTY,
KEY_64,
KEY_STRING,
HASHED,
};
Type type = Type::EMPTY;
static Type chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes);
bool keys_fit_128_bits; bool keys_fit_128_bits;
Sizes key_sizes; Sizes key_sizes;
@ -174,7 +183,7 @@ private:
*/ */
mutable Poco::RWLock rwlock; mutable Poco::RWLock rwlock;
void init(Set::Type type_); void init(Type type_);
template <ASTJoin::Strictness STRICTNESS, typename Maps> template <ASTJoin::Strictness STRICTNESS, typename Maps>
void insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainPtrs & key_columns, size_t keys_size, Block * stored_block); void insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainPtrs & key_columns, size_t keys_size, Block * stored_block);

View File

@ -24,6 +24,245 @@ namespace DB
{ {
/** Методы для разных вариантов реализации множеств.
* Используются в качестве параметра шаблона.
*/
/// Для случая, когда есть один числовой ключ.
template <typename FieldType, typename TData> /// UInt8/16/32/64 для любых типов соответствующей битности.
struct SetMethodOneNumber
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
/// Для использования одного Method в разных потоках, используйте разные State.
struct State
{
const FieldType * vec;
/** Вызывается в начале обработки каждого блока.
* Устанавливает переменные, необходимые для остальных методов, вызываемых во внутренних циклах.
*/
void init(const ConstColumnPlainPtrs & key_columns)
{
vec = &static_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData()[0];
}
/// Достать из ключевых столбцов ключ для вставки в хэш-таблицу.
Key getKey(
const ConstColumnPlainPtrs & key_columns, /// Ключевые столбцы.
size_t keys_size, /// Количество ключевых столбцов.
size_t i, /// Из какой строки блока достать ключ.
const Sizes & key_sizes) const /// Если ключи фиксированной длины - их длины. Не используется в методах по ключам переменной длины.
{
return unionCastToUInt64(vec[i]);
}
};
/** Разместить дополнительные данные, если это необходимо, в случае, когда в хэш-таблицу был вставлен новый ключ.
*/
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
};
/// Для случая, когда есть один строковый ключ.
template <typename TData>
struct SetMethodString
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
struct State
{
const ColumnString::Offsets_t * offsets;
const ColumnString::Chars_t * chars;
void init(const ConstColumnPlainPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnString & column_string = static_cast<const ColumnString &>(column);
offsets = &column_string.getOffsets();
chars = &column_string.getChars();
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
return StringRef(
&(*chars)[i == 0 ? 0 : (*offsets)[i - 1]],
(i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool)
{
value.data = pool.insert(value.data, value.size);
}
};
/// Для случая, когда есть один строковый ключ фиксированной длины.
template <typename TData>
struct SetMethodFixedString
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
struct State
{
size_t n;
const ColumnFixedString::Chars_t * chars;
void init(const ConstColumnPlainPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnFixedString & column_string = static_cast<const ColumnFixedString &>(column);
n = column_string.getN();
chars = &column_string.getChars();
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
return StringRef(&(*chars)[i * n], n);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool)
{
value.data = pool.insert(value.data, value.size);
}
};
/// Для случая, когда все ключи фиксированной длины, и они помещаются в N (например, 128) бит.
template <typename TData>
struct SetMethodKeysFixed
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
struct State
{
void init(const ConstColumnPlainPtrs & key_columns)
{
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
};
/// Для остальных случаев. По 128 битному хэшу от ключа. (При этом, строки, содержащие нули посередине, могут склеиться.)
template <typename TData>
struct SetMethodHashed
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
struct State
{
void init(const ConstColumnPlainPtrs & key_columns)
{
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
return hash128(i, keys_size, key_columns);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
};
/** Разные варианты реализации множества.
*/
struct SetVariants
{
/// TODO Использовать для этих двух вариантов bit- или byte- set.
std::unique_ptr<SetMethodOneNumber<UInt8, HashSet<UInt8, TrivialHash, HashTableFixedGrower<8>>>> key8;
std::unique_ptr<SetMethodOneNumber<UInt16, HashSet<UInt16, TrivialHash, HashTableFixedGrower<16>>>> key16;
/** Также для эксперимента проверялась возможность использовать SmallSet,
* пока количество элементов в множестве небольшое (и, при необходимости, конвертировать в полноценный HashSet).
* Но этот эксперимент показал, что преимущество есть только в редких случаях.
*/
std::unique_ptr<SetMethodOneNumber<UInt32, HashSet<UInt32, HashCRC32<UInt32>>>> key32;
std::unique_ptr<SetMethodOneNumber<UInt64, HashSet<UInt64, HashCRC32<UInt64>>>> key64;
std::unique_ptr<SetMethodString<HashSetWithSavedHash<StringRef>>> key_string;
std::unique_ptr<SetMethodFixedString<HashSetWithSavedHash<StringRef>>> key_fixed_string;
std::unique_ptr<SetMethodKeysFixed<HashSet<UInt128, UInt128HashCRC32>>> keys128;
std::unique_ptr<SetMethodKeysFixed<HashSet<UInt256, UInt256HashCRC32>>> keys256;
std::unique_ptr<SetMethodHashed<HashSet<UInt128, UInt128TrivialHash>>> hashed;
/** В отличие от Aggregator, здесь не используется метод concat.
* Это сделано потому что метод hashed, хоть и медленнее, но в данном случае, использует меньше оперативки.
* так как при его использовании, сами значения ключей не сохраняются.
*/
Arena string_pool;
#define APPLY_FOR_SET_VARIANTS(M) \
M(key8) \
M(key16) \
M(key32) \
M(key64) \
M(key_string) \
M(key_fixed_string) \
M(keys128) \
M(keys256) \
M(hashed)
enum class Type
{
EMPTY,
#define M(NAME) NAME,
APPLY_FOR_SET_VARIANTS(M)
#undef M
};
Type type = Type::EMPTY;
bool empty() const { return type == Type::EMPTY; }
static Type chooseMethod(const ConstColumnPlainPtrs & key_columns, Sizes & key_sizes);
void init(Type type_);
size_t getTotalRowCount() const;
/// Считает размер в байтах буфера Set и размер string_pool'а
size_t getTotalByteCount() const;
};
/** Структура данных для реализации выражения IN. /** Структура данных для реализации выражения IN.
*/ */
class Set class Set
@ -37,7 +276,7 @@ public:
{ {
} }
bool empty() { return type == EMPTY; } bool empty() const { return data.empty(); }
/** Создать множество по выражению (для перечисления в самом запросе). /** Создать множество по выражению (для перечисления в самом запросе).
* types - типы того, что стоит слева от IN. * types - типы того, что стоит слева от IN.
@ -49,11 +288,6 @@ public:
// Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять. // Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
bool insertFromBlock(const Block & block, bool create_ordered_set = false); bool insertFromBlock(const Block & block, bool create_ordered_set = false);
/// Считает суммарное число ключей во всех Set'ах
size_t getTotalRowCount() const;
/// Считает суммарный размер в байтах буфферов всех Set'ов + размер string_pool'а
size_t getTotalByteCount() const;
/** Для указанных столбцов блока проверить принадлежность их значений множеству. /** Для указанных столбцов блока проверить принадлежность их значений множеству.
* Записать результат в столбец в позиции result. * Записать результат в столбец в позиции result.
*/ */
@ -80,43 +314,14 @@ public:
/// проверяет есть ли в Set элементы для заданного диапазона индекса /// проверяет есть ли в Set элементы для заданного диапазона индекса
BoolMask mayBeTrueInRange(const Range & range); BoolMask mayBeTrueInRange(const Range & range);
enum Type size_t getTotalRowCount() const { return data.getTotalRowCount(); }
{ size_t getTotalByteCount() const { return data.getTotalByteCount(); }
EMPTY = 0,
KEY_64 = 1,
KEY_STRING = 2,
HASHED = 3,
};
static Type chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes);
private: private:
/** Разные структуры данных, которые могут использоваться для проверки принадлежности
* одного или нескольких столбцов значений множеству.
*/
typedef HashSet<UInt64, HashCRC32<UInt64>> SetUInt64;
typedef HashSetWithSavedHash<StringRef> SetString;
typedef HashSet<UInt128, UInt128HashCRC32> SetHashed;
/// Специализация для случая, когда есть один числовой ключ.
std::unique_ptr<SetUInt64> key64;
/// Специализация для случая, когда есть один строковый ключ.
std::unique_ptr<SetString> key_string;
Arena string_pool;
/** Сравнивает 128 битные хэши.
* Если все ключи фиксированной длины, влезающие целиком в 128 бит, то укладывает их без изменений в 128 бит.
* Иначе - вычисляет SipHash от набора из всех ключей.
* (При этом, строки, содержащие нули посередине, могут склеиться.)
*/
std::unique_ptr<SetHashed> hashed;
Type type = EMPTY;
bool keys_fit_128_bits;
Sizes key_sizes; Sizes key_sizes;
SetVariants data;
/** Типы данных, из которых было создано множество. /** Типы данных, из которых было создано множество.
* При проверке на принадлежность множеству, типы проверяемых столбцов должны с ними совпадать. * При проверке на принадлежность множеству, типы проверяемых столбцов должны с ними совпадать.
*/ */
@ -129,24 +334,7 @@ private:
size_t max_bytes; size_t max_bytes;
OverflowMode overflow_mode; OverflowMode overflow_mode;
void init(Type type_)
{
type = type_;
switch (type)
{
case EMPTY: break;
case KEY_64: key64 .reset(new SetUInt64); break;
case KEY_STRING: key_string .reset(new SetString); break;
case HASHED: hashed .reset(new SetHashed); break;
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
}
/// Если в левой части IN стоит массив. Проверяем, что хоть один элемент массива лежит в множестве. /// Если в левой части IN стоит массив. Проверяем, что хоть один элемент массива лежит в множестве.
void executeConstArray(const ColumnConstArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const;
void executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const; void executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const;
/// Если в левой части набор столбцов тех же типов, что элементы множества. /// Если в левой части набор столбцов тех же типов, что элементы множества.
@ -155,8 +343,8 @@ private:
/// Проверить не превышены ли допустимые размеры множества ключей /// Проверить не превышены ли допустимые размеры множества ключей
bool checkSetSizeLimits() const; bool checkSetSizeLimits() const;
/// вектор упорядоченных элементов Set /// Вектор упорядоченных элементов Set.
/// нужен для работы индекса по первичному ключу в секции In /// Нужен для работы индекса по первичному ключу в операторе IN.
typedef std::vector<Field> OrderedSetElements; typedef std::vector<Field> OrderedSetElements;
typedef std::unique_ptr<OrderedSetElements> OrderedSetElementsPtr; typedef std::unique_ptr<OrderedSetElements> OrderedSetElementsPtr;
OrderedSetElementsPtr ordered_set_elements; OrderedSetElementsPtr ordered_set_elements;
@ -167,6 +355,31 @@ private:
* Поэтому остальные функции по работе с множеством, не защинены. * Поэтому остальные функции по работе с множеством, не защинены.
*/ */
mutable Poco::RWLock rwlock; mutable Poco::RWLock rwlock;
template <typename Method>
void insertFromBlockImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
size_t rows,
SetVariants & variants);
template <typename Method>
void executeImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const;
template <typename Method>
void executeArrayImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
const ColumnArray::Offsets_t & offsets,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const;
}; };
typedef Poco::SharedPtr<Set> SetPtr; typedef Poco::SharedPtr<Set> SetPtr;

View File

@ -108,7 +108,7 @@ struct Settings
M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \ M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \
\ \
/** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \ /** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \
M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 4) \ M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \
/// Всевозможные ограничения на выполнение запроса. /// Всевозможные ограничения на выполнение запроса.
Limits limits; Limits limits;

View File

@ -140,7 +140,6 @@ void AggregateFunctionsCreator::operator()()
for (size_t rollback_j = 0; rollback_j < column_num; ++rollback_j) for (size_t rollback_j = 0; rollback_j < column_num; ++rollback_j)
func->destroy(aggregate_data + offsets_of_aggregate_states[rollback_j]); func->destroy(aggregate_data + offsets_of_aggregate_states[rollback_j]);
aggregate_data = nullptr;
throw; throw;
} }
} }
@ -239,10 +238,14 @@ void NO_INLINE Aggregator::executeSpecializedCase(
method.onNewKey(*it, keys_size, i, keys, *aggregates_pool); method.onNewKey(*it, keys_size, i, keys, *aggregates_pool);
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = aggregates_pool->alloc(total_size_of_aggregate_states);
aggregate_data = nullptr;
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
AggregateFunctionsList::forEach(AggregateFunctionsCreator( AggregateFunctionsList::forEach(AggregateFunctionsCreator(
aggregate_functions, offsets_of_aggregate_states, aggregate_columns, aggregate_data)); aggregate_functions, offsets_of_aggregate_states, aggregate_columns, place));
aggregate_data = place;
} }
else else
method.onExistingKey(key, keys, *aggregates_pool); method.onExistingKey(key, keys, *aggregates_pool);

View File

@ -2,32 +2,8 @@
#include <ostream> #include <ostream>
#include <DB/Core/NamesAndTypes.h>
#include <DB/Parsers/IAST.h> #include <DB/Parsers/IAST.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTDropQuery.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTColumnDeclaration.h>
#include <DB/Parsers/ASTAsterisk.h>
#include <DB/Parsers/ASTOrderByElement.h>
#include <DB/Parsers/ASTSubquery.h>
#include <DB/Parsers/ASTAlterQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/Parsers/ASTJoin.h>
#include <DB/Parsers/ASTCheckQuery.h>
//#include <DB/Parsers/ASTMultiQuery.h>
namespace DB namespace DB
@ -38,39 +14,6 @@ namespace DB
*/ */
void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTColumnDeclaration & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTSet & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTJoin & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTCheckQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
//void formatAST(const ASTMultiQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::ostream & s,
size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTShowProcesslistQuery & ast, std::ostream & s,
size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
String formatColumnsForCreateQuery(NamesAndTypesList & columns); String formatColumnsForCreateQuery(NamesAndTypesList & columns);
String backQuoteIfNeed(const String & x); String backQuoteIfNeed(const String & x);

View File

@ -154,6 +154,80 @@ static IAggregateFunction * createAggregateFunctionSingleValue(const String & na
} }
/// argMin, argMax
template <template <typename> class MinMaxData, typename ResData>
static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const String & name, const IDataType & val_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt8>>>>;
else if (typeid_cast<const DataTypeUInt16 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt16>>>>;
else if (typeid_cast<const DataTypeUInt32 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt32>>>>;
else if (typeid_cast<const DataTypeUInt64 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt64>>>>;
else if (typeid_cast<const DataTypeInt8 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int8>>>>;
else if (typeid_cast<const DataTypeInt16 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int16>>>>;
else if (typeid_cast<const DataTypeInt32 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int32>>>>;
else if (typeid_cast<const DataTypeInt64 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int64>>>>;
else if (typeid_cast<const DataTypeFloat32 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Float32>>>>;
else if (typeid_cast<const DataTypeFloat64 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Float64>>>>;
else if (typeid_cast<const DataTypeDate *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDate::FieldType>>>>;
else if (typeid_cast<const DataTypeDateTime*>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDateTime::FieldType>>>>;
else if (typeid_cast<const DataTypeString*>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataString>>>;
else
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataGeneric>>>;
}
template <template <typename> class MinMaxData>
static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & res_type = *argument_types[0];
const IDataType & val_type = *argument_types[1];
if (typeid_cast<const DataTypeUInt8 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt8>>(name, val_type);
else if (typeid_cast<const DataTypeUInt16 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt16>>(name, val_type);
else if (typeid_cast<const DataTypeUInt32 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt32>>(name, val_type);
else if (typeid_cast<const DataTypeUInt64 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt64>>(name, val_type);
else if (typeid_cast<const DataTypeInt8 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int8>>(name, val_type);
else if (typeid_cast<const DataTypeInt16 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int16>>(name, val_type);
else if (typeid_cast<const DataTypeInt32 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int32>>(name, val_type);
else if (typeid_cast<const DataTypeInt64 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int64>>(name, val_type);
else if (typeid_cast<const DataTypeFloat32 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Float32>>(name, val_type);
else if (typeid_cast<const DataTypeFloat64 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Float64>>(name, val_type);
else if (typeid_cast<const DataTypeDate *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDate::FieldType>>(name, val_type);
else if (typeid_cast<const DataTypeDateTime*>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDateTime::FieldType>>(name, val_type);
else if (typeid_cast<const DataTypeString*>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataString>(name, val_type);
else
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataGeneric>(name, val_type);
}
AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const DataTypes & argument_types, int recursion_level) const AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const DataTypes & argument_types, int recursion_level) const
{ {
if (name == "count") if (name == "count")
@ -167,9 +241,9 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
else if (name == "max") else if (name == "max")
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types); return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types);
else if (name == "argMin") else if (name == "argMin")
return new AggregateFunctionArgMin; return createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types);
else if (name == "argMax") else if (name == "argMax")
return new AggregateFunctionArgMax; return createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types);
else if (name == "groupArray") else if (name == "groupArray")
return new AggregateFunctionGroupArray; return new AggregateFunctionGroupArray;
else if (name == "groupUniqArray") else if (name == "groupUniqArray")

View File

@ -40,6 +40,12 @@
#include <DB/DataStreams/AsynchronousBlockInputStream.h> #include <DB/DataStreams/AsynchronousBlockInputStream.h>
#include <DB/Parsers/ParserQuery.h> #include <DB/Parsers/ParserQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTQueryWithOutput.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/formatAST.h> #include <DB/Parsers/formatAST.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -359,7 +365,7 @@ private:
bool ends_with_semicolon = line[ws - 1] == ';'; bool ends_with_semicolon = line[ws - 1] == ';';
bool ends_with_backslash = line[ws - 1] == '\\'; bool ends_with_backslash = line[ws - 1] == '\\';
has_vertical_output_suffix = (ws >= 2) && (line[ws - 2] == '\\') && (line[ws - 1] == 'G'); has_vertical_output_suffix = (ws >= 2) && (line[ws - 2] == '\\') && (line[ws - 1] == 'G');
if (ends_with_backslash) if (ends_with_backslash)
@ -373,7 +379,7 @@ private:
{ {
// Заменяем переводы строк на пробелы, а то возникает следуцющая проблема. // Заменяем переводы строк на пробелы, а то возникает следуцющая проблема.
// Каждая строчка многострочного запроса сохраняется в истории отдельно. Если // Каждая строчка многострочного запроса сохраняется в истории отдельно. Если
// выйти из клиента и войти заново, то при нажатии клавиши "вверх" выводится не // выйти из клиента и войти заново, то при нажатии клавиши "вверх" выводится не
// весь многострочный запрос, а каждая его строчка по-отдельности. // весь многострочный запрос, а каждая его строчка по-отдельности.
std::string logged_query = query; std::string logged_query = query;
std::replace(logged_query.begin(), logged_query.end(), '\n', ' '); std::replace(logged_query.begin(), logged_query.end(), '\n', ' ');
@ -799,7 +805,7 @@ private:
onException(*packet.exception); onException(*packet.exception);
last_exception = packet.exception; last_exception = packet.exception;
return false; return false;
default: default:
throw Exception("Unexpected packet from server (expected Data, got " throw Exception("Unexpected packet from server (expected Data, got "
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
@ -819,7 +825,7 @@ private:
if (!block_std_out) if (!block_std_out)
{ {
String current_format = format; String current_format = format;
/// Формат может быть указан в запросе. /// Формат может быть указан в запросе.
if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query)) if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query))
{ {
@ -831,10 +837,10 @@ private:
current_format = id->name; current_format = id->name;
} }
} }
if (has_vertical_output_suffix) if (has_vertical_output_suffix)
current_format = "Vertical"; current_format = "Vertical";
block_std_out = context.getFormatFactory().getOutput(current_format, std_out, block); block_std_out = context.getFormatFactory().getOutput(current_format, std_out, block);
block_std_out->writePrefix(); block_std_out->writePrefix();
} }

View File

@ -185,7 +185,7 @@ std::string ParallelReplicas::dumpAddresses() const
if (connection != nullptr) if (connection != nullptr)
{ {
os << (is_first ? "" : "; ") << connection->getServerAddress(); os << (is_first ? "" : "; ") << connection->getServerAddress();
if (is_first) { is_first = false; } is_first = false;
} }
} }

File diff suppressed because one or more lines are too long

View File

@ -1,3 +1,5 @@
#include <DB/Interpreters/Set.h>
#include <DB/Interpreters/Join.h>
#include <DB/DataStreams/CreatingSetsBlockInputStream.h> #include <DB/DataStreams/CreatingSetsBlockInputStream.h>
#include <iomanip> #include <iomanip>

View File

@ -25,6 +25,8 @@
#include <DB/Parsers/ParserSelectQuery.h> #include <DB/Parsers/ParserSelectQuery.h>
#include <DB/Parsers/formatAST.h> #include <DB/Parsers/formatAST.h>
#include <DB/Interpreters/Context.h>
using Poco::SharedPtr; using Poco::SharedPtr;

View File

@ -109,8 +109,8 @@ namespace VisibleWidth
if (const ColumnVector<T> * col = typeid_cast<const ColumnVector<T> *>(&*column)) if (const ColumnVector<T> * col = typeid_cast<const ColumnVector<T> *>(&*column))
{ {
ColumnUInt64 * res = new ColumnUInt64(column->size()); ColumnUInt64 * res = new ColumnUInt64(column->size());
numWidthVector(col->getData(), res->getData());
block.getByPosition(result).column = res; block.getByPosition(result).column = res;
numWidthVector(col->getData(), res->getData());
return true; return true;
} }
else else
@ -158,20 +158,20 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
else if (const ColumnString * col = typeid_cast<const ColumnString *>(&*column)) else if (const ColumnString * col = typeid_cast<const ColumnString *>(&*column))
{ {
ColumnUInt64 * res = new ColumnUInt64(rows); ColumnUInt64 * res = new ColumnUInt64(rows);
stringWidthVector(col->getChars(), col->getOffsets(), res->getData());
block.getByPosition(result).column = res; block.getByPosition(result).column = res;
stringWidthVector(col->getChars(), col->getOffsets(), res->getData());
} }
else if (const ColumnFixedString * col = typeid_cast<const ColumnFixedString *>(&*column)) else if (const ColumnFixedString * col = typeid_cast<const ColumnFixedString *>(&*column))
{ {
ColumnUInt64 * res = new ColumnUInt64(rows); ColumnUInt64 * res = new ColumnUInt64(rows);
stringWidthFixedVector(col->getChars(), col->getN(), res->getData());
block.getByPosition(result).column = res; block.getByPosition(result).column = res;
stringWidthFixedVector(col->getChars(), col->getN(), res->getData());
} }
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*column)) else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*column))
{ {
UInt64 res = 0; UInt64 res = 0;
stringWidthConstant(col->getData(), res);
block.getByPosition(result).column = new ColumnConstUInt64(rows, res); block.getByPosition(result).column = new ColumnConstUInt64(rows, res);
stringWidthConstant(col->getData(), res);
} }
else if (const ColumnArray * col = typeid_cast<const ColumnArray *>(&*column)) else if (const ColumnArray * col = typeid_cast<const ColumnArray *>(&*column))
{ {
@ -191,6 +191,7 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
/// Теперь суммируем и кладём в результат. /// Теперь суммируем и кладём в результат.
ColumnUInt64 * res = new ColumnUInt64(rows); ColumnUInt64 * res = new ColumnUInt64(rows);
block.getByPosition(result).column = res;
ColumnUInt64::Container_t & vec = res->getData(); ColumnUInt64::Container_t & vec = res->getData();
size_t additional_symbols = 0; /// Кавычки. size_t additional_symbols = 0; /// Кавычки.
@ -223,8 +224,6 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
vec[i] = 1 + std::max(static_cast<size_t>(1), vec[i] = 1 + std::max(static_cast<size_t>(1),
(i == 0 ? col->getOffsets()[0] : (col->getOffsets()[i] - col->getOffsets()[i - 1])) * nested_length); (i == 0 ? col->getOffsets()[0] : (col->getOffsets()[i] - col->getOffsets()[i - 1])) * nested_length);
} }
block.getByPosition(result).column = res;
} }
else if (const ColumnTuple * col = typeid_cast<const ColumnTuple *>(&*column)) else if (const ColumnTuple * col = typeid_cast<const ColumnTuple *>(&*column))
{ {

View File

@ -381,7 +381,6 @@ void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const
for (size_t rollback_j = 0; rollback_j < j; ++rollback_j) for (size_t rollback_j = 0; rollback_j < j; ++rollback_j)
aggregate_functions[rollback_j]->destroy(aggregate_data + offsets_of_aggregate_states[rollback_j]); aggregate_functions[rollback_j]->destroy(aggregate_data + offsets_of_aggregate_states[rollback_j]);
aggregate_data = nullptr;
throw; throw;
} }
} }
@ -486,8 +485,12 @@ void NO_INLINE Aggregator::executeImplCase(
method.onNewKey(*it, keys_size, i, keys, *aggregates_pool); method.onNewKey(*it, keys_size, i, keys, *aggregates_pool);
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(aggregate_data); /// exception-safety - если не удалось выделить память или создать состояния, то не будут вызываться деструкторы.
aggregate_data = nullptr;
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(place);
aggregate_data = place;
} }
else else
method.onExistingKey(key, keys, *aggregates_pool); method.onExistingKey(key, keys, *aggregates_pool);
@ -581,8 +584,9 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
if ((overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key) if ((overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key)
{ {
result.without_key = result.aggregates_pool->alloc(total_size_of_aggregate_states); AggregateDataPtr place = result.aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(result.without_key); createAggregateStates(place);
result.without_key = place;
} }
/// Выбираем один из методов агрегации и вызываем его. /// Выбираем один из методов агрегации и вызываем его.
@ -831,6 +835,12 @@ Block Aggregator::prepareBlockAndFill(
} }
filler(key_columns, aggregate_columns, final_aggregate_columns, data_variants.key_sizes, final); filler(key_columns, aggregate_columns, final_aggregate_columns, data_variants.key_sizes, final);
/// Изменяем размер столбцов-констант в блоке.
size_t columns = res.columns();
for (size_t i = 0; i < columns; ++i)
if (res.getByPosition(i).column->isConst())
res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows);
} }
catch (...) catch (...)
{ {
@ -848,12 +858,6 @@ Block Aggregator::prepareBlockAndFill(
throw; throw;
} }
/// Изменяем размер столбцов-констант в блоке.
size_t columns = res.columns();
for (size_t i = 0; i < columns; ++i)
if (res.getByPosition(i).column->isConst())
res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows);
return res; return res;
} }
@ -1035,7 +1039,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, bool final, size_t max_threads) BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, bool final, size_t max_threads)
{ {
LOG_TRACE(log, "Converting aggregated data to block"); LOG_TRACE(log, "Converting aggregated data to blocks");
Stopwatch watch; Stopwatch watch;
@ -1365,8 +1369,11 @@ void NO_INLINE Aggregator::mergeStreamsImpl(
method.onNewKey(*it, keys_size, i, keys, *aggregates_pool); method.onNewKey(*it, keys_size, i, keys, *aggregates_pool);
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(aggregate_data); aggregate_data = nullptr;
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(place);
aggregate_data = place;
} }
else else
method.onExistingKey(key, keys, *aggregates_pool); method.onExistingKey(key, keys, *aggregates_pool);
@ -1395,8 +1402,9 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
AggregatedDataWithoutKey & res = result.without_key; AggregatedDataWithoutKey & res = result.without_key;
if (!res) if (!res)
{ {
res = result.aggregates_pool->alloc(total_size_of_aggregate_states); AggregateDataPtr place = result.aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(res); createAggregateStates(place);
res = place;
} }
/// Добавляем значения /// Добавляем значения

View File

@ -43,6 +43,8 @@ void ExternalDictionaries::reloadImpl()
const auto last_modified = config_file.getLastModified(); const auto last_modified = config_file.getLastModified();
if (last_modified > config_last_modified) if (last_modified > config_last_modified)
{ {
stored_exceptions.clear();
/// definitions of dictionaries may have changed, recreate all of them /// definitions of dictionaries may have changed, recreate all of them
config_last_modified = last_modified; config_last_modified = last_modified;
@ -58,6 +60,8 @@ void ExternalDictionaries::reloadImpl()
/// for each dictionary defined in xml config /// for each dictionary defined in xml config
for (const auto & key : keys) for (const auto & key : keys)
{ {
std::string name;
try try
{ {
if (0 != strncmp(key.data(), "dictionary", strlen("dictionary"))) if (0 != strncmp(key.data(), "dictionary", strlen("dictionary")))
@ -66,7 +70,7 @@ void ExternalDictionaries::reloadImpl()
continue; continue;
} }
const auto name = config->getString(key + ".name"); name = config->getString(key + ".name");
if (name.empty()) if (name.empty())
{ {
LOG_WARNING(log, "dictionary name cannot be empty"); LOG_WARNING(log, "dictionary name cannot be empty");
@ -100,7 +104,25 @@ void ExternalDictionaries::reloadImpl()
} }
catch (...) catch (...)
{ {
handleException(); if (!name.empty())
stored_exceptions.emplace(name, std::current_exception());
try
{
throw;
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "Cannot load external dictionary! You must resolve this manually. " << e.displayText());
}
catch (const std::exception & e)
{
LOG_ERROR(log, "Cannot load external dictionary! You must resolve this manually. " << e.what());
}
catch (...)
{
LOG_ERROR(log, "Cannot load external dictionary! You must resolve this manually.");
}
} }
} }
} }
@ -142,9 +164,20 @@ void ExternalDictionaries::reloadImpl()
} }
} }
} }
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "Cannot update external dictionary '" << dictionary.first
<< "'! You must resolve this manually. " << e.displayText());
}
catch (const std::exception & e)
{
LOG_ERROR(log, "Cannot load external dictionary '" << dictionary.first
<< "'! You must resolve this manually. " << e.what());
}
catch (...) catch (...)
{ {
handleException(); LOG_ERROR(log, "Cannot update external dictionary '" << dictionary.first
<< "'! You must resolve this manually.");
} }
} }
} }

View File

@ -5,6 +5,7 @@
#include <DB/Parsers/ASTExpressionList.h> #include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTNameTypePair.h> #include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTIdentifier.h> #include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ParserCreateQuery.h> #include <DB/Parsers/ParserCreateQuery.h>
#include <DB/IO/copyData.h> #include <DB/IO/copyData.h>

View File

@ -8,6 +8,7 @@
#include <DB/Parsers/ASTSetQuery.h> #include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h> #include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/ASTAlterQuery.h> #include <DB/Parsers/ASTAlterQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h> #include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTCheckQuery.h> #include <DB/Parsers/ASTCheckQuery.h>

View File

@ -8,6 +8,7 @@
#include <DB/IO/copyData.h> #include <DB/IO/copyData.h>
#include <DB/Parsers/ASTRenameQuery.h> #include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ParserCreateQuery.h> #include <DB/Parsers/ParserCreateQuery.h>
#include <DB/Parsers/formatAST.h> #include <DB/Parsers/formatAST.h>

View File

@ -12,7 +12,6 @@
#include <DB/DataStreams/DistinctBlockInputStream.h> #include <DB/DataStreams/DistinctBlockInputStream.h>
#include <DB/DataStreams/NullBlockInputStream.h> #include <DB/DataStreams/NullBlockInputStream.h>
#include <DB/DataStreams/TotalsHavingBlockInputStream.h> #include <DB/DataStreams/TotalsHavingBlockInputStream.h>
#include <DB/DataStreams/narrowBlockInputStreams.h>
#include <DB/DataStreams/copyData.h> #include <DB/DataStreams/copyData.h>
#include <DB/DataStreams/CreatingSetsBlockInputStream.h> #include <DB/DataStreams/CreatingSetsBlockInputStream.h>
#include <DB/DataStreams/MaterializingBlockInputStream.h> #include <DB/DataStreams/MaterializingBlockInputStream.h>
@ -697,12 +696,6 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu
streams.push_back(interpreter_subquery->execute()); streams.push_back(interpreter_subquery->execute());
} }
/** Если истчоников слишком много, то склеим их в max_threads источников.
* (Иначе действия в каждом маленьком источнике, а затем объединение состояний, слишком неэффективно.)
*/
if (streams.size() > settings.max_threads)
streams = narrowBlockInputStreams(streams, settings.max_threads);
/** Установка ограничений и квоты на чтение данных, скорость и время выполнения запроса. /** Установка ограничений и квоты на чтение данных, скорость и время выполнения запроса.
* Такие ограничения проверяются на сервере-инициаторе запроса, а не на удалённых серверах. * Такие ограничения проверяются на сервере-инициаторе запроса, а не на удалённых серверах.
* Потому что сервер-инициатор имеет суммарные данные о выполнении запроса на всех серверах. * Потому что сервер-инициатор имеет суммарные данные о выполнении запроса на всех серверах.

View File

@ -9,15 +9,53 @@ namespace DB
{ {
Join::Type Join::chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes)
{
size_t keys_size = key_columns.size();
keys_fit_128_bits = true;
size_t keys_bytes = 0;
key_sizes.resize(keys_size);
for (size_t j = 0; j < keys_size; ++j)
{
if (!key_columns[j]->isFixed())
{
keys_fit_128_bits = false;
break;
}
key_sizes[j] = key_columns[j]->sizeOfField();
keys_bytes += key_sizes[j];
}
if (keys_bytes > 16)
keys_fit_128_bits = false;
/// Если есть один числовой ключ, который помещается в 64 бита
if (keys_size == 1 && key_columns[0]->isNumeric())
return Type::KEY_64;
/// Если есть один строковый ключ, то используем хэш-таблицу с ним
if (keys_size == 1
&& (typeid_cast<const ColumnString *>(key_columns[0])
|| typeid_cast<const ColumnConstString *>(key_columns[0])
|| (typeid_cast<const ColumnFixedString *>(key_columns[0]) && !keys_fit_128_bits)))
return Type::KEY_STRING;
/// Если много ключей - будем строить множество хэшей от них
return Type::HASHED;
}
template <typename Maps> template <typename Maps>
static void initImpl(Maps & maps, Set::Type type) static void initImpl(Maps & maps, Join::Type type)
{ {
switch (type) switch (type)
{ {
case Set::EMPTY: break; case Join::Type::EMPTY: break;
case Set::KEY_64: maps.key64 .reset(new typename Maps::MapUInt64); break; case Join::Type::KEY_64: maps.key64 .reset(new typename Maps::MapUInt64); break;
case Set::KEY_STRING: maps.key_string .reset(new typename Maps::MapString); break; case Join::Type::KEY_STRING: maps.key_string .reset(new typename Maps::MapString); break;
case Set::HASHED: maps.hashed .reset(new typename Maps::MapHashed); break; case Join::Type::HASHED: maps.hashed .reset(new typename Maps::MapHashed); break;
default: default:
throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
@ -51,7 +89,7 @@ static size_t getTotalByteCountImpl(const Maps & maps)
} }
void Join::init(Set::Type type_) void Join::init(Type type_)
{ {
type = type_; type = type_;
@ -190,7 +228,7 @@ struct Inserter<ASTJoin::All, Join::MapsAll::MapString>
template <ASTJoin::Strictness STRICTNESS, typename Maps> template <ASTJoin::Strictness STRICTNESS, typename Maps>
void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainPtrs & key_columns, size_t keys_size, Block * stored_block) void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainPtrs & key_columns, size_t keys_size, Block * stored_block)
{ {
if (type == Set::KEY_64) if (type == Type::KEY_64)
{ {
typedef typename Maps::MapUInt64 Map; typedef typename Maps::MapUInt64 Map;
Map & res = *maps.key64; Map & res = *maps.key64;
@ -204,7 +242,7 @@ void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainP
Inserter<STRICTNESS, Map>::insert(res, key, stored_block, i, pool); Inserter<STRICTNESS, Map>::insert(res, key, stored_block, i, pool);
} }
} }
else if (type == Set::KEY_STRING) else if (type == Type::KEY_STRING)
{ {
typedef typename Maps::MapString Map; typedef typename Maps::MapString Map;
Map & res = *maps.key_string; Map & res = *maps.key_string;
@ -239,7 +277,7 @@ void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainP
else else
throw Exception("Illegal type of column when creating join with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN); throw Exception("Illegal type of column when creating join with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
} }
else if (type == Set::HASHED) else if (type == Type::HASHED)
{ {
typedef typename Maps::MapHashed Map; typedef typename Maps::MapHashed Map;
Map & res = *maps.hashed; Map & res = *maps.hashed;
@ -274,7 +312,7 @@ bool Join::insertFromBlock(const Block & block)
/// Какую структуру данных для множества использовать? /// Какую структуру данных для множества использовать?
if (empty()) if (empty())
init(Set::chooseMethod(key_columns, keys_fit_128_bits, key_sizes)); init(chooseMethod(key_columns, keys_fit_128_bits, key_sizes));
blocks.push_back(block); blocks.push_back(block);
Block * stored_block = &blocks.back(); Block * stored_block = &blocks.back();
@ -441,7 +479,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
if (strictness == ASTJoin::All) if (strictness == ASTJoin::All)
offsets_to_replicate.reset(new IColumn::Offsets_t(rows)); offsets_to_replicate.reset(new IColumn::Offsets_t(rows));
if (type == Set::KEY_64) if (type == Type::KEY_64)
{ {
typedef typename Maps::MapUInt64 Map; typedef typename Maps::MapUInt64 Map;
const Map & map = *maps.key64; const Map & map = *maps.key64;
@ -455,7 +493,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
Adder<KIND, STRICTNESS, Map>::add(map, key, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get()); Adder<KIND, STRICTNESS, Map>::add(map, key, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get());
} }
} }
else if (type == Set::KEY_STRING) else if (type == Type::KEY_STRING)
{ {
typedef typename Maps::MapString Map; typedef typename Maps::MapString Map;
const Map & map = *maps.key_string; const Map & map = *maps.key_string;
@ -490,7 +528,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
else else
throw Exception("Illegal type of column when creating set with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN); throw Exception("Illegal type of column when creating set with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
} }
else if (type == Set::HASHED) else if (type == Type::HASHED)
{ {
typedef typename Maps::MapHashed Map; typedef typename Maps::MapHashed Map;
Map & map = *maps.hashed; Map & map = *maps.hashed;

View File

@ -21,78 +21,144 @@
namespace DB namespace DB
{ {
size_t Set::getTotalRowCount() const void SetVariants::init(Type type_)
{ {
size_t rows = 0; type = type_;
if (key64)
rows += key64->size(); switch (type)
if (key_string) {
rows += key_string->size(); case Type::EMPTY: break;
if (hashed)
rows += hashed->size(); #define M(NAME) \
return rows; case Type::NAME: NAME.reset(new decltype(NAME)::element_type); break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
} }
size_t Set::getTotalByteCount() const size_t SetVariants::getTotalRowCount() const
{ {
size_t bytes = 0; switch (type)
if (key64) {
bytes += key64->getBufferSizeInBytes(); case Type::EMPTY: return 0;
if (key_string)
bytes += key_string->getBufferSizeInBytes(); #define M(NAME) \
if (hashed) case Type::NAME: return NAME->data.size();
bytes += hashed->getBufferSizeInBytes(); APPLY_FOR_SET_VARIANTS(M)
bytes += string_pool.size(); #undef M
return bytes;
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
}
size_t SetVariants::getTotalByteCount() const
{
switch (type)
{
case Type::EMPTY: return 0;
#define M(NAME) \
case Type::NAME: return NAME->data.getBufferSizeInBytes();
APPLY_FOR_SET_VARIANTS(M)
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
} }
bool Set::checkSetSizeLimits() const bool Set::checkSetSizeLimits() const
{ {
if (max_rows && getTotalRowCount() > max_rows) if (max_rows && data.getTotalRowCount() > max_rows)
return false; return false;
if (max_bytes && getTotalByteCount() > max_bytes) if (max_bytes && data.getTotalByteCount() > max_bytes)
return false; return false;
return true; return true;
} }
Set::Type Set::chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes) SetVariants::Type SetVariants::chooseMethod(const ConstColumnPlainPtrs & key_columns, Sizes & key_sizes)
{ {
size_t keys_size = key_columns.size(); size_t keys_size = key_columns.size();
keys_fit_128_bits = true; bool all_fixed = true;
size_t keys_bytes = 0; size_t keys_bytes = 0;
key_sizes.resize(keys_size); key_sizes.resize(keys_size);
for (size_t j = 0; j < keys_size; ++j) for (size_t j = 0; j < keys_size; ++j)
{ {
if (!key_columns[j]->isFixed()) if (!key_columns[j]->isFixed())
{ {
keys_fit_128_bits = false; all_fixed = false;
break; break;
} }
key_sizes[j] = key_columns[j]->sizeOfField(); key_sizes[j] = key_columns[j]->sizeOfField();
keys_bytes += key_sizes[j]; keys_bytes += key_sizes[j];
} }
if (keys_bytes > 16)
keys_fit_128_bits = false;
/// Если есть один числовой ключ, который помещается в 64 бита /// Если есть один числовой ключ, который помещается в 64 бита
if (keys_size == 1 && key_columns[0]->isNumeric()) if (keys_size == 1 && key_columns[0]->isNumeric())
return KEY_64; {
size_t size_of_field = key_columns[0]->sizeOfField();
if (size_of_field == 1)
return SetVariants::Type::key8;
if (size_of_field == 2)
return SetVariants::Type::key16;
if (size_of_field == 4)
return SetVariants::Type::key32;
if (size_of_field == 8)
return SetVariants::Type::key64;
throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8.", ErrorCodes::LOGICAL_ERROR);
}
/// Если ключи помещаются в N бит, будем использовать хэш-таблицу по упакованным в N-бит ключам
if (all_fixed && keys_bytes <= 16)
return SetVariants::Type::keys128;
if (all_fixed && keys_bytes <= 32)
return SetVariants::Type::keys256;
/// Если есть один строковый ключ, то используем хэш-таблицу с ним /// Если есть один строковый ключ, то используем хэш-таблицу с ним
if (keys_size == 1 if (keys_size == 1 && (typeid_cast<const ColumnString *>(key_columns[0]) || typeid_cast<const ColumnConstString *>(key_columns[0])))
&& (typeid_cast<const ColumnString *>(key_columns[0]) return SetVariants::Type::key_string;
|| typeid_cast<const ColumnConstString *>(key_columns[0])
|| (typeid_cast<const ColumnFixedString *>(key_columns[0]) && !keys_fit_128_bits)))
return KEY_STRING;
/// Если много ключей - будем строить множество хэшей от них if (keys_size == 1 && typeid_cast<const ColumnFixedString *>(key_columns[0]))
return HASHED; return SetVariants::Type::key_fixed_string;
/// Иначе будем агрегировать по конкатенации ключей.
return SetVariants::Type::hashed;
}
template <typename Method>
void NO_INLINE Set::insertFromBlockImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
size_t rows,
SetVariants & variants)
{
typename Method::State state;
state.init(key_columns);
size_t keys_size = key_columns.size();
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes);
typename Method::Data::iterator it = method.data.find(key);
bool inserted;
method.data.emplace(key, it, inserted);
if (inserted)
method.onNewKey(*it, keys_size, i, variants.string_pool);
}
} }
@ -115,94 +181,28 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
/// Какую структуру данных для множества использовать? /// Какую структуру данных для множества использовать?
if (empty()) if (empty())
init(chooseMethod(key_columns, keys_fit_128_bits, key_sizes)); data.init(data.chooseMethod(key_columns, key_sizes));
if (type == KEY_64) if (false) {}
{ #define M(NAME) \
SetUInt64 & res = *key64; else if (data.type == SetVariants::Type::NAME) \
const IColumn & column = *key_columns[0]; insertFromBlockImpl(*data.NAME, key_columns, rows, data);
APPLY_FOR_SET_VARIANTS(M)
/// Для всех строчек #undef M
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
UInt64 key = column.get64(i);
res.insert(key);
if(create_ordered_set)
ordered_set_elements->push_back(column[i]);
}
}
else if (type == KEY_STRING)
{
SetString & res = *key_string;
const IColumn & column = *key_columns[0];
if (const ColumnString * column_string = typeid_cast<const ColumnString *>(&column))
{
const ColumnString::Offsets_t & offsets = column_string->getOffsets();
const ColumnString::Chars_t & data = column_string->getChars();
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
StringRef ref(&data[i == 0 ? 0 : offsets[i - 1]], (i == 0 ? offsets[i] : (offsets[i] - offsets[i - 1])) - 1);
SetString::iterator it;
bool inserted;
res.emplace(ref, it, inserted);
if (inserted)
it->data = string_pool.insert(ref.data, ref.size);
if(create_ordered_set)
ordered_set_elements->push_back(std::string(ref.data, ref.size));
}
}
else if (const ColumnFixedString * column_string = typeid_cast<const ColumnFixedString *>(&column))
{
size_t n = column_string->getN();
const ColumnFixedString::Chars_t & data = column_string->getChars();
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
StringRef ref(&data[i * n], n);
SetString::iterator it;
bool inserted;
res.emplace(ref, it, inserted);
if (inserted)
it->data = string_pool.insert(ref.data, ref.size);
if(create_ordered_set)
ordered_set_elements->push_back(std::string(ref.data, ref.size));
}
}
else
throw Exception("Illegal type of column when creating set with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else if (type == HASHED)
{
SetHashed & res = *hashed;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
res.insert(keys_fit_128_bits ? packFixed<UInt128>(i, keys_size, key_columns, key_sizes) : hash128(i, keys_size, key_columns));
}
else else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT); throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
if (create_ordered_set)
for (size_t i = 0; i < rows; ++i)
ordered_set_elements->push_back((*key_columns[0])[i]); /// ordered_set для индекса работает только если IN одному ключу.
if (!checkSetSizeLimits()) if (!checkSetSizeLimits())
{ {
if (overflow_mode == OverflowMode::THROW) if (overflow_mode == OverflowMode::THROW)
throw Exception("IN-Set size exceeded." throw Exception("IN-set size exceeded."
" Rows: " + toString(getTotalRowCount()) + " Rows: " + toString(data.getTotalRowCount()) +
", limit: " + toString(max_rows) + ", limit: " + toString(max_rows) +
". Bytes: " + toString(getTotalByteCount()) + ". Bytes: " + toString(data.getTotalByteCount()) +
", limit: " + toString(max_bytes) + ".", ", limit: " + toString(max_bytes) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
@ -417,12 +417,19 @@ void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result,
throw Exception(std::string() + "Types in section IN don't match: " + data_types[0]->getName() + " on the right, " + array_type->getNestedType()->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH); throw Exception(std::string() + "Types in section IN don't match: " + data_types[0]->getName() + " on the right, " + array_type->getNestedType()->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH);
IColumn * in_column = &*block.getByPosition(arguments[0]).column; IColumn * in_column = &*block.getByPosition(arguments[0]).column;
if (ColumnConstArray * col = typeid_cast<ColumnConstArray *>(in_column))
executeConstArray(col, vec_res, negative); /// Константный столбец слева от IN поддерживается не напрямую. Для этого, он сначала материализуется.
else if (ColumnArray * col = typeid_cast<ColumnArray *>(in_column)) ColumnPtr materialized_column;
if (in_column->isConst())
{
materialized_column = static_cast<const IColumnConst *>(in_column)->convertToFullColumn();
in_column = materialized_column.get();
}
if (ColumnArray * col = typeid_cast<ColumnArray *>(in_column))
executeArray(col, vec_res, negative); executeArray(col, vec_res, negative);
else else
throw Exception("Unexpeced array column type: " + in_column->getName(), ErrorCodes::ILLEGAL_COLUMN); throw Exception("Unexpected array column type: " + in_column->getName(), ErrorCodes::ILLEGAL_COLUMN);
} }
else else
{ {
@ -439,82 +446,88 @@ void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result,
throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: " + data_types[i]->getName() + " on the right, " + block.getByPosition(arguments[i]).type->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH); throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: " + data_types[i]->getName() + " on the right, " + block.getByPosition(arguments[i]).type->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH);
} }
/// Константные столбцы слева от IN поддерживается не напрямую. Для этого, они сначала материализуется.
Columns materialized_columns;
for (auto & column_ptr : key_columns)
{
if (column_ptr->isConst())
{
materialized_columns.emplace_back(static_cast<const IColumnConst *>(column_ptr)->convertToFullColumn());
column_ptr = materialized_columns.back().get();
}
}
executeOrdinary(key_columns, vec_res, negative); executeOrdinary(key_columns, vec_res, negative);
} }
} }
void Set::executeOrdinary(const ConstColumnPlainPtrs & key_columns, ColumnUInt8::Container_t & vec_res, bool negative) const
template <typename Method>
void NO_INLINE Set::executeImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const
{ {
size_t keys_size = data_types.size(); typename Method::State state;
size_t rows = key_columns[0]->size(); state.init(key_columns);
Row key(keys_size); size_t keys_size = key_columns.size();
if (type == KEY_64) /// NOTE Не используется оптимизация для подряд идущих одинаковых значений.
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{ {
const SetUInt64 & set = *key64; /// Строим ключ
const IColumn & column = *key_columns[0]; typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes);
vec_res[i] = negative ^ (method.data.end() != method.data.find(key));
}
}
/// Для всех строчек template <typename Method>
for (size_t i = 0; i < rows; ++i) void NO_INLINE Set::executeArrayImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
const ColumnArray::Offsets_t & offsets,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const
{
typename Method::State state;
state.init(key_columns);
size_t keys_size = key_columns.size();
size_t prev_offset = 0;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{ {
/// Строим ключ /// Строим ключ
UInt64 key = column.get64(i); typename Method::Key key = state.getKey(key_columns, keys_size, j, key_sizes);
vec_res[i] = negative ^ (set.end() != set.find(key)); res |= negative ^ (method.data.end() != method.data.find(key));
if (res)
break;
} }
vec_res[i] = res;
prev_offset = offsets[i];
} }
else if (type == KEY_STRING) }
{
const SetString & set = *key_string;
const IColumn & column = *key_columns[0];
if (const ColumnString * column_string = typeid_cast<const ColumnString *>(&column))
{
const ColumnString::Offsets_t & offsets = column_string->getOffsets();
const ColumnString::Chars_t & data = column_string->getChars();
/// Для всех строчек void Set::executeOrdinary(const ConstColumnPlainPtrs & key_columns, ColumnUInt8::Container_t & vec_res, bool negative) const
for (size_t i = 0; i < rows; ++i) {
{ size_t rows = key_columns[0]->size();
/// Строим ключ
StringRef ref(&data[i == 0 ? 0 : offsets[i - 1]], (i == 0 ? offsets[i] : (offsets[i] - offsets[i - 1])) - 1);
vec_res[i] = negative ^ (set.end() != set.find(ref));
}
}
else if (const ColumnFixedString * column_string = typeid_cast<const ColumnFixedString *>(&column))
{
size_t n = column_string->getN();
const ColumnFixedString::Chars_t & data = column_string->getChars();
/// Для всех строчек if (false) {}
for (size_t i = 0; i < rows; ++i) #define M(NAME) \
{ else if (data.type == SetVariants::Type::NAME) \
/// Строим ключ executeImpl(*data.NAME, key_columns, vec_res, negative, rows);
StringRef ref(&data[i * n], n); APPLY_FOR_SET_VARIANTS(M)
vec_res[i] = negative ^ (set.end() != set.find(ref)); #undef M
}
}
else if (const ColumnConstString * column_string = typeid_cast<const ColumnConstString *>(&column))
{
bool res = negative ^ (set.end() != set.find(StringRef(column_string->getData())));
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
vec_res[i] = res;
}
else
throw Exception("Illegal type of column when creating set with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else if (type == HASHED)
{
const SetHashed & set = *hashed;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
vec_res[i] = negative ^
(set.end() != set.find(keys_fit_128_bits
? packFixed<UInt128>(i, keys_size, key_columns, key_sizes)
: hash128(i, keys_size, key_columns)));
}
else else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT); throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
} }
@ -525,150 +538,16 @@ void Set::executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t
const ColumnArray::Offsets_t & offsets = key_column->getOffsets(); const ColumnArray::Offsets_t & offsets = key_column->getOffsets();
const IColumn & nested_column = key_column->getData(); const IColumn & nested_column = key_column->getData();
if (type == KEY_64) if (false) {}
{ #define M(NAME) \
const SetUInt64 & set = *key64; else if (data.type == SetVariants::Type::NAME) \
executeArrayImpl(*data.NAME, ConstColumnPlainPtrs{&nested_column}, offsets, vec_res, negative, rows);
size_t prev_offset = 0; APPLY_FOR_SET_VARIANTS(M)
/// Для всех строчек #undef M
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
UInt64 key = nested_column.get64(j);
res |= negative ^ (set.end() != set.find(key));
if (res)
break;
}
vec_res[i] = res;
prev_offset = offsets[i];
}
}
else if (type == KEY_STRING)
{
const SetString & set = *key_string;
if (const ColumnString * column_string = typeid_cast<const ColumnString *>(&nested_column))
{
const ColumnString::Offsets_t & nested_offsets = column_string->getOffsets();
const ColumnString::Chars_t & data = column_string->getChars();
size_t prev_offset = 0;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
size_t begin = j == 0 ? 0 : nested_offsets[j - 1];
size_t end = nested_offsets[j];
StringRef ref(&data[begin], end - begin - 1);
res |= negative ^ (set.end() != set.find(ref));
if (res)
break;
}
vec_res[i] = res;
prev_offset = offsets[i];
}
}
else if (const ColumnFixedString * column_string = typeid_cast<const ColumnFixedString *>(&nested_column))
{
size_t n = column_string->getN();
const ColumnFixedString::Chars_t & data = column_string->getChars();
size_t prev_offset = 0;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
StringRef ref(&data[j * n], n);
res |= negative ^ (set.end() != set.find(ref));
if (res)
break;
}
vec_res[i] = res;
prev_offset = offsets[i];
}
}
else
throw Exception("Illegal type of column when looking for Array(String) key: " + nested_column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else if (type == HASHED)
{
const SetHashed & set = *hashed;
ConstColumnPlainPtrs nested_columns(1, &nested_column);
size_t prev_offset = 0;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
res |= negative ^
(set.end() != set.find(keys_fit_128_bits
? packFixed<UInt128>(i, 1, nested_columns, key_sizes)
: hash128(i, 1, nested_columns)));
if (res)
break;
}
vec_res[i] = res;
prev_offset = offsets[i];
}
}
else else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT); throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
} }
void Set::executeConstArray(const ColumnConstArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const
{
if (type == HASHED)
{
ColumnPtr full_column = key_column->convertToFullColumn();
executeArray(typeid_cast<ColumnArray *>(&*full_column), vec_res, negative);
return;
}
size_t rows = key_column->size();
Array values = key_column->getData();
UInt8 res = 0;
/// Для всех элементов
for (size_t j = 0; j < values.size(); ++j)
{
if (type == KEY_64)
{
const SetUInt64 & set = *key64;
UInt64 key = get<UInt64>(values[j]);
res |= negative ^ (set.end() != set.find(key));
}
else if (type == KEY_STRING)
{
const SetString & set = *key_string;
res |= negative ^ (set.end() != set.find(StringRef(get<String>(values[j]))));
}
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
if (res)
break;
}
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
vec_res[i] = res;
}
BoolMask Set::mayBeTrueInRange(const Range & range) BoolMask Set::mayBeTrueInRange(const Range & range)
{ {

View File

@ -3,6 +3,8 @@
#include <DB/Parsers/formatAST.h> #include <DB/Parsers/formatAST.h>
#include <DB/DataStreams/BlockIO.h> #include <DB/DataStreams/BlockIO.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Interpreters/executeQuery.h> #include <DB/Interpreters/executeQuery.h>

View File

@ -63,6 +63,16 @@ struct AlternativeHash
} }
}; };
struct CRC32Hash_
{
size_t operator() (UInt64 x) const
{
UInt64 crc = -1ULL;
asm("crc32q %[x], %[crc]\n" : [crc] "+r" (crc) : [x] "rm" (x));
return crc;
}
};
int main(int argc, char ** argv) int main(int argc, char ** argv)
{ {
@ -81,7 +91,7 @@ int main(int argc, char ** argv)
DB::DataTypes data_types_empty; DB::DataTypes data_types_empty;
DB::DataTypes data_types_uint64; DB::DataTypes data_types_uint64;
data_types_uint64.push_back(new DB::DataTypeUInt64); data_types_uint64.push_back(new DB::DataTypeUInt64);
std::vector<Key> data(n); std::vector<Key> data(n);
Value value; Value value;
@ -181,7 +191,7 @@ int main(int argc, char ** argv)
watch.stop(); watch.stop();
std::cerr << std::fixed << std::setprecision(2) std::cerr << std::fixed << std::setprecision(2)
<< "HashMap. Size: " << map.size() << "HashMap, AlternativeHash. Size: " << map.size()
<< ", elapsed: " << watch.elapsedSeconds() << ", elapsed: " << watch.elapsedSeconds()
<< " (" << n / watch.elapsedSeconds() << " elem/sec.)" << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS #ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
@ -194,6 +204,36 @@ int main(int argc, char ** argv)
{ {
Stopwatch watch; Stopwatch watch;
typedef HashMap<Key, Value, CRC32Hash_> Map;
Map map;
Map::iterator it;
bool inserted;
for (size_t i = 0; i < n; ++i)
{
map.emplace(data[i], it, inserted);
if (inserted)
{
new(&it->second) Value(std::move(value));
INIT;
}
}
watch.stop();
std::cerr << std::fixed << std::setprecision(2)
<< "HashMap, CRC32Hash. Size: " << map.size()
<< ", elapsed: " << watch.elapsedSeconds()
<< " (" << n / watch.elapsedSeconds() << " elem/sec.)"
#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
<< ", collisions: " << map.getCollisions()
#endif
<< std::endl;
}
if (argc < 3 || atoi(argv[2]) == 4)
{
Stopwatch watch;
std::unordered_map<Key, Value, DefaultHash<Key> > map; std::unordered_map<Key, Value, DefaultHash<Key> > map;
std::unordered_map<Key, Value, DefaultHash<Key> >::iterator it; std::unordered_map<Key, Value, DefaultHash<Key> >::iterator it;
for (size_t i = 0; i < n; ++i) for (size_t i = 0; i < n; ++i)
@ -210,7 +250,7 @@ int main(int argc, char ** argv)
<< std::endl; << std::endl;
} }
if (argc < 3 || atoi(argv[2]) == 4) if (argc < 3 || atoi(argv[2]) == 5)
{ {
Stopwatch watch; Stopwatch watch;
@ -231,7 +271,7 @@ int main(int argc, char ** argv)
<< std::endl; << std::endl;
} }
if (argc < 3 || atoi(argv[2]) == 5) if (argc < 3 || atoi(argv[2]) == 6)
{ {
Stopwatch watch; Stopwatch watch;
@ -250,6 +290,6 @@ int main(int argc, char ** argv)
<< " (" << n / watch.elapsedSeconds() << " elem/sec.)" << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
<< std::endl; << std::endl;
} }
return 0; return 0;
} }

View File

@ -440,7 +440,7 @@ inline bool memequal_sse_wide(const char * p1, const char * p2, size_t size)
if ( compare_byIntSSE(p1, p2) if ( compare_byIntSSE(p1, p2)
&& compare_byIntSSE(p1 + 16, p2 + 16) && compare_byIntSSE(p1 + 16, p2 + 16)
&& compare_byIntSSE(p1 + 32, p2 + 32) && compare_byIntSSE(p1 + 32, p2 + 32)
&& compare_byIntSSE(p1 + 40, p2 + 40)) && compare_byIntSSE(p1 + 48, p2 + 48))
{ {
p1 += 64; p1 += 64;
p2 += 64; p2 += 64;

View File

@ -1,7 +1,5 @@
#include <sstream> #include <sstream>
#include <boost/variant/static_visitor.hpp>
#include <mysqlxx/Manip.h> #include <mysqlxx/Manip.h>
#include <DB/IO/WriteBufferFromOStream.h> #include <DB/IO/WriteBufferFromOStream.h>
@ -12,6 +10,32 @@
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Core/NamesAndTypes.h> #include <DB/Core/NamesAndTypes.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTDropQuery.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTColumnDeclaration.h>
#include <DB/Parsers/ASTAsterisk.h>
#include <DB/Parsers/ASTOrderByElement.h>
#include <DB/Parsers/ASTSubquery.h>
#include <DB/Parsers/ASTAlterQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/Parsers/ASTJoin.h>
#include <DB/Parsers/ASTCheckQuery.h>
//#include <DB/Parsers/ASTMultiQuery.h>
#include <DB/Parsers/formatAST.h> #include <DB/Parsers/formatAST.h>
@ -45,52 +69,6 @@ String hightlight(const String & keyword, const String & color_sequence, const b
} }
void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
#define DISPATCH(NAME) \
else if (const AST ## NAME * concrete = typeid_cast<const AST ## NAME *>(&ast)) \
formatAST(*concrete, s, indent, hilite, one_line, need_parens);
if (false) {}
DISPATCH(SelectQuery)
DISPATCH(InsertQuery)
DISPATCH(CreateQuery)
DISPATCH(DropQuery)
DISPATCH(RenameQuery)
DISPATCH(ShowTablesQuery)
DISPATCH(UseQuery)
DISPATCH(SetQuery)
DISPATCH(OptimizeQuery)
DISPATCH(ExistsQuery)
DISPATCH(ShowCreateQuery)
DISPATCH(DescribeQuery)
DISPATCH(ExpressionList)
DISPATCH(Function)
DISPATCH(Identifier)
DISPATCH(Literal)
DISPATCH(NameTypePair)
DISPATCH(ColumnDeclaration)
DISPATCH(Asterisk)
DISPATCH(OrderByElement)
DISPATCH(Subquery)
DISPATCH(AlterQuery)
DISPATCH(ShowProcesslistQuery)
DISPATCH(Set)
DISPATCH(Join)
DISPATCH(CheckQuery)
// DISPATCH(MultiQuery)
else
throw Exception("Unknown element in AST: " + ast.getID()
+ ((ast.range.first && (ast.range.second > ast.range.first))
? " '" + std::string(ast.range.first, ast.range.second - ast.range.first) + "'"
: ""),
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
#undef DISPATCH
}
void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{ {
for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it) for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it)
@ -354,17 +332,17 @@ void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::os
void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{ {
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "EXISTS TABLE", s, indent, hilite, one_line); formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "EXISTS TABLE", s, indent, hilite, one_line, false);
} }
void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{ {
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "DESCRIBE TABLE", s, indent, hilite, one_line); formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "DESCRIBE TABLE", s, indent, hilite, one_line, false);
} }
void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{ {
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "SHOW CREATE TABLE", s, indent, hilite, one_line); formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "SHOW CREATE TABLE", s, indent, hilite, one_line, false);
} }
void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
@ -888,6 +866,52 @@ void formatAST(const ASTMultiQuery & ast, std::ostream & s, size_t indent, bool
}*/ }*/
void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
#define DISPATCH(NAME) \
else if (const AST ## NAME * concrete = typeid_cast<const AST ## NAME *>(&ast)) \
formatAST(*concrete, s, indent, hilite, one_line, need_parens);
if (false) {}
DISPATCH(SelectQuery)
DISPATCH(InsertQuery)
DISPATCH(CreateQuery)
DISPATCH(DropQuery)
DISPATCH(RenameQuery)
DISPATCH(ShowTablesQuery)
DISPATCH(UseQuery)
DISPATCH(SetQuery)
DISPATCH(OptimizeQuery)
DISPATCH(ExistsQuery)
DISPATCH(ShowCreateQuery)
DISPATCH(DescribeQuery)
DISPATCH(ExpressionList)
DISPATCH(Function)
DISPATCH(Identifier)
DISPATCH(Literal)
DISPATCH(NameTypePair)
DISPATCH(ColumnDeclaration)
DISPATCH(Asterisk)
DISPATCH(OrderByElement)
DISPATCH(Subquery)
DISPATCH(AlterQuery)
DISPATCH(ShowProcesslistQuery)
DISPATCH(Set)
DISPATCH(Join)
DISPATCH(CheckQuery)
// DISPATCH(MultiQuery)
else
throw Exception("Unknown element in AST: " + ast.getID()
+ ((ast.range.first && (ast.range.second > ast.range.first))
? " '" + std::string(ast.range.first, ast.range.second - ast.range.first) + "'"
: ""),
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
#undef DISPATCH
}
String formatColumnsForCreateQuery(NamesAndTypesList & columns) String formatColumnsForCreateQuery(NamesAndTypesList & columns)
{ {
std::string res; std::string res;

View File

@ -54,15 +54,6 @@ struct QueryState
void reset() void reset()
{ {
/** process_list_entry также включает/выключает учёт памяти MemoryTracker-ом.
* Члены maybe_compressed_in, block_in, maybe_compressed_out, block_out
* могли быть инициализированы до io, и выделенная в них память могла не быть учтена MemoryTracker-ом.
* Если эти члены будут уничтожены раньше, то освобождение памяти будет учтено MemoryTracker-ом,
* и вычисленный расход памяти может оказаться отрицательным (это не проблема, но некрасиво).
* Поэтому, сначала уничтожим process_list_entry.
*/
io.process_list_entry = nullptr;
*this = QueryState(); *this = QueryState();
} }

View File

@ -4,7 +4,6 @@
#include <DB/Parsers/ASTIdentifier.h> #include <DB/Parsers/ASTIdentifier.h>
#include <DB/DataStreams/ExpressionBlockInputStream.h> #include <DB/DataStreams/ExpressionBlockInputStream.h>
#include <DB/DataStreams/FilterBlockInputStream.h> #include <DB/DataStreams/FilterBlockInputStream.h>
#include <DB/DataStreams/ConcatBlockInputStream.h>
#include <DB/DataStreams/CollapsingFinalBlockInputStream.h> #include <DB/DataStreams/CollapsingFinalBlockInputStream.h>
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h> #include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
#include <DB/DataStreams/CreatingSetsBlockInputStream.h> #include <DB/DataStreams/CreatingSetsBlockInputStream.h>
@ -374,7 +373,6 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads(
for (size_t i = 0; i < threads && !parts.empty(); ++i) for (size_t i = 0; i < threads && !parts.empty(); ++i)
{ {
size_t need_marks = min_marks_per_thread; size_t need_marks = min_marks_per_thread;
BlockInputStreams streams;
/// Цикл по кускам. /// Цикл по кускам.
while (need_marks > 0 && !parts.empty()) while (need_marks > 0 && !parts.empty())
@ -427,7 +425,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads(
} }
} }
streams.push_back(new MergeTreeBlockInputStream( res.push_back(new MergeTreeBlockInputStream(
data.getFullPath() + part.data_part->name + '/', max_block_size, column_names, data, data.getFullPath() + part.data_part->name + '/', max_block_size, column_names, data,
part.data_part, ranges_to_get_from_part, use_uncompressed_cache, part.data_part, ranges_to_get_from_part, use_uncompressed_cache,
prewhere_actions, prewhere_column)); prewhere_actions, prewhere_column));
@ -435,18 +433,13 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads(
for (const String & virt_column : virt_columns) for (const String & virt_column : virt_columns)
{ {
if (virt_column == "_part") if (virt_column == "_part")
streams.back() = new AddingConstColumnBlockInputStream<String>( res.back() = new AddingConstColumnBlockInputStream<String>(
streams.back(), new DataTypeString, part.data_part->name, "_part"); res.back(), new DataTypeString, part.data_part->name, "_part");
else if (virt_column == "_part_index") else if (virt_column == "_part_index")
streams.back() = new AddingConstColumnBlockInputStream<UInt64>( res.back() = new AddingConstColumnBlockInputStream<UInt64>(
streams.back(), new DataTypeUInt64, part.part_index_in_query, "_part_index"); res.back(), new DataTypeUInt64, part.part_index_in_query, "_part_index");
} }
} }
if (streams.size() == 1)
res.push_back(streams[0]);
else
res.push_back(new ConcatBlockInputStream(streams));
} }
if (!parts.empty()) if (!parts.empty())

View File

@ -12,7 +12,6 @@
#include <DB/Interpreters/executeQuery.h> #include <DB/Interpreters/executeQuery.h>
#include <DB/Interpreters/InterpreterDropQuery.h> #include <DB/Interpreters/InterpreterDropQuery.h>
#include <DB/DataStreams/ConcatBlockInputStream.h> #include <DB/DataStreams/ConcatBlockInputStream.h>
#include <DB/DataStreams/narrowBlockInputStreams.h>
#include <DB/DataStreams/AddingDefaultBlockInputStream.h> #include <DB/DataStreams/AddingDefaultBlockInputStream.h>
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h> #include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
#include <DB/Common/VirtualColumnUtils.h> #include <DB/Common/VirtualColumnUtils.h>
@ -210,11 +209,6 @@ BlockInputStreams StorageChunkMerger::read(
processed_stage = tmp_processed_stage; processed_stage = tmp_processed_stage;
} }
/** Если истчоников слишком много, то склеим их в threads источников.
*/
if (res.size() > threads)
res = narrowBlockInputStreams(res, threads);
return res; return res;
} }

View File

@ -6,6 +6,7 @@
#include <DB/Storages/Distributed/DistributedBlockOutputStream.h> #include <DB/Storages/Distributed/DistributedBlockOutputStream.h>
#include <DB/Storages/Distributed/DirectoryMonitor.h> #include <DB/Storages/Distributed/DirectoryMonitor.h>
#include <DB/Common/escapeForFileName.h> #include <DB/Common/escapeForFileName.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Interpreters/InterpreterSelectQuery.h> #include <DB/Interpreters/InterpreterSelectQuery.h>
#include <DB/Interpreters/InterpreterAlterQuery.h> #include <DB/Interpreters/InterpreterAlterQuery.h>

View File

@ -1,4 +1,3 @@
#include <DB/DataStreams/narrowBlockInputStreams.h>
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h> #include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
#include <DB/Storages/StorageMerge.h> #include <DB/Storages/StorageMerge.h>
#include <DB/Common/VirtualColumnUtils.h> #include <DB/Common/VirtualColumnUtils.h>
@ -173,11 +172,6 @@ BlockInputStreams StorageMerge::read(
processed_stage = tmp_processed_stage; processed_stage = tmp_processed_stage;
} }
/** Если истчоников слишком много, то склеим их в threads источников.
*/
if (res.size() > threads)
res = narrowBlockInputStreams(res, threads);
return res; return res;
} }

View File

@ -8,6 +8,7 @@
#include <DB/IO/ReadBufferFromString.h> #include <DB/IO/ReadBufferFromString.h>
#include <DB/Interpreters/InterpreterAlterQuery.h> #include <DB/Interpreters/InterpreterAlterQuery.h>
#include <DB/Common/VirtualColumnUtils.h> #include <DB/Common/VirtualColumnUtils.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h> #include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
#include <time.h> #include <time.h>

View File

@ -0,0 +1 @@
1

View File

@ -0,0 +1 @@
SELECT (1, '') IN ((1, ''));

View File

@ -0,0 +1,27 @@
1
1
1
1
1
1
1
0
0
1
0
0
1
0
0
1
0
1
0
1
1
0
0
1
1
0
1

View File

@ -0,0 +1,21 @@
SELECT 1 IN (1, 2, 3);
SELECT toUInt16(1) IN (1, 1000, 3);
SELECT [1, 2, 3] IN (3, 4, 5);
SELECT materialize([1, 2, 3]) IN (3, 4, 5);
SELECT 'Hello' IN ('Hello', 'world');
SELECT (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) IN ((1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17));
SELECT (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, '') IN ((1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, ''));
SELECT (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, '') IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 'a');
SELECT (number AS n, n + 1, n + 2, n + 3) IN (1, 2, 3, 4) FROM system.numbers LIMIT 3;
SELECT (number AS n, n + 1, n + 2, n + 3, n - 1) IN (1, 2, 3, 4, 0) FROM system.numbers LIMIT 3;
SELECT (number AS n, n + 1, toString(n + 2), n + 3, n - 1) IN (1, 2, '3', 4, 0) FROM system.numbers LIMIT 3;
SELECT [1, 2, 3] IN (2);
SELECT [1, 2, 3] IN (4);
SELECT [1, 2, 3] NOT IN (1);
SELECT [1, 2, 3] NOT IN (1, 2);
SELECT [1, 2, 3] NOT IN (1, 2, 3);
SELECT [1, 2, 3] NOT IN (1, 2, 3, 4);
SELECT ['Hello', 'world'] IN ('world');
SELECT ['Hello', 'world'] NOT IN ('world');
SELECT ['Hello', 'world'] NOT IN ('Hello', 'world');
SELECT ['Hello', 'world'] NOT IN ('hello', 'world');

View File

@ -0,0 +1,16 @@
#!/bin/bash
clickhouse-client -n --query="
DROP TABLE IF EXISTS test.numbers_100k;
CREATE VIEW test.numbers_100k AS SELECT * FROM system.numbers LIMIT 100000;
";
for i in $(seq 1000000 20000 10000000 && seq 10100000 100000 20000000); do
clickhouse-client --max_memory_usage=$i --query="
SELECT intDiv(number, 5) AS k, max(toString(number)) FROM remote('127.0.0.{1,2}', test.numbers_100k) GROUP BY k ORDER BY k LIMIT 1;
" 2> /dev/null;
CODE=$?;
[ "$CODE" -ne "241" ] && [ "$CODE" -ne "0" ] && echo "Fail" && break;
done | uniq
clickhouse-client --query="DROP TABLE test.numbers_100k;";

View File

@ -13,7 +13,7 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & cfg
cfg.keys(config_name, replica_keys); cfg.keys(config_name, replica_keys);
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = replica_keys.begin(); it != replica_keys.end(); ++it) for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = replica_keys.begin(); it != replica_keys.end(); ++it)
{ {
if (!(*it == "port" || *it == "user" || *it == "password" || *it == "db")) if (!(*it == "port" || *it == "user" || *it == "password" || *it == "db" || *it == "table"))
{ {
if (it->size() < std::string("replica").size() || it->substr(0, std::string("replica").size()) != "replica") if (it->size() < std::string("replica").size() || it->substr(0, std::string("replica").size()) != "replica")
throw Poco::Exception("Unknown element in config: " + *it + ", expected replica"); throw Poco::Exception("Unknown element in config: " + *it + ", expected replica");