This commit is contained in:
Pavel Kartavyy 2015-07-22 12:22:32 +03:00
commit e93918d48b
178 changed files with 15996 additions and 1003 deletions

View File

@ -15,6 +15,7 @@
#include <DB/Interpreters/AggregationCommon.h> #include <DB/Interpreters/AggregationCommon.h>
#include <DB/Common/HashTable/HashSet.h> #include <DB/Common/HashTable/HashSet.h>
#include <DB/Common/HyperLogLogWithSmallSetOptimization.h> #include <DB/Common/HyperLogLogWithSmallSetOptimization.h>
#include <DB/Common/CombinedCardinalityEstimator.h>
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>
@ -117,6 +118,26 @@ struct AggregateFunctionUniqExactData<String>
}; };
template <typename T>
struct AggregateFunctionUniqCombinedData
{
using Key = T;
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, DefaultHash<Key>, HashTableGrower<4> >, 16, 16, 19>;
Set set;
static String getName() { return "uniqCombined"; }
};
template <>
struct AggregateFunctionUniqCombinedData<String>
{
using Key = UInt64;
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, DefaultHash<Key>, HashTableGrower<4> >, 16, 16, 19>;
Set set;
static String getName() { return "uniqCombined"; }
};
namespace detail namespace detail
{ {
/** Структура для делегации работы по добавлению одного элемента в агрегатные функции uniq. /** Структура для делегации работы по добавлению одного элемента в агрегатные функции uniq.
@ -166,6 +187,28 @@ namespace detail
data.set.insert(key); data.set.insert(key);
} }
}; };
template<typename T>
struct OneAdder<T, AggregateFunctionUniqCombinedData<T> >
{
static void addOne(AggregateFunctionUniqCombinedData<T> & data, const IColumn & column, size_t row_num)
{
if (data.set.isMedium())
data.set.insert(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
else
data.set.insert(AggregateFunctionUniqTraits<T>::hash(static_cast<const ColumnVector<T> &>(column).getData()[row_num]));
}
};
template<>
struct OneAdder<String, AggregateFunctionUniqCombinedData<String> >
{
static void addOne(AggregateFunctionUniqCombinedData<String> & data, const IColumn & column, size_t row_num)
{
StringRef value = column.getDataAt(row_num);
data.set.insert(CityHash64(value.data, value.size));
}
};
} }

View File

@ -0,0 +1,262 @@
#pragma once
#include <DB/Common/HashTable/SmallTable.h>
#include <DB/Common/HashTable/HashSet.h>
#include <DB/Common/HyperLogLogWithSmallSetOptimization.h>
namespace DB
{
namespace details
{
enum class ContainerType { SMALL, MEDIUM, LARGE };
ContainerType max(const ContainerType & lhs, const ContainerType & rhs)
{
unsigned int res = std::max(static_cast<unsigned int>(lhs), static_cast<unsigned int>(rhs));
return static_cast<ContainerType>(res);
}
}
/** Для маленького количества ключей - массив фиксированного размера "на стеке".
* Для среднего - выделяется HashSet.
* Для большого - выделяется HyperLogLog.
*/
template <typename Key, typename HashContainer, UInt8 small_set_size_max, UInt8 medium_set_power2_max, UInt8 K>
class CombinedCardinalityEstimator
{
public:
using Self = CombinedCardinalityEstimator<Key, HashContainer, small_set_size_max, medium_set_power2_max, K>;
private:
using Small = SmallSet<Key, small_set_size_max>;
using Medium = HashContainer;
using Large = HyperLogLogWithSmallSetOptimization<Key, small_set_size_max, K>;
public:
~CombinedCardinalityEstimator()
{
if (container_type == details::ContainerType::MEDIUM)
{
delete medium;
if (current_memory_tracker)
current_memory_tracker->free(sizeof(medium));
}
else if (container_type == details::ContainerType::LARGE)
{
delete large;
if (current_memory_tracker)
current_memory_tracker->free(sizeof(large));
}
}
void insert(Key value)
{
if (container_type == details::ContainerType::SMALL)
{
if (small.find(value) == small.end())
{
if (!small.full())
small.insert(value);
else
{
toMedium();
medium->insert(value);
}
}
}
else if (container_type == details::ContainerType::MEDIUM)
{
if (medium->size() < medium_set_size_max)
medium->insert(value);
else
{
toLarge();
large->insert(value);
}
}
else if (container_type == details::ContainerType::LARGE)
large->insert(value);
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
}
UInt32 size() const
{
if (container_type == details::ContainerType::SMALL)
return small.size();
else if (container_type == details::ContainerType::MEDIUM)
return medium->size();
else if (container_type == details::ContainerType::LARGE)
return large->size();
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
}
void merge(const Self & rhs)
{
details::ContainerType max_container_type = details::max(container_type, rhs.container_type);
if (container_type != max_container_type)
{
if (max_container_type == details::ContainerType::MEDIUM)
toMedium();
else if (max_container_type == details::ContainerType::LARGE)
toLarge();
}
if (container_type == details::ContainerType::SMALL)
{
for (const auto & x : rhs.small)
insert(x);
}
else if (container_type == details::ContainerType::MEDIUM)
{
if (rhs.container_type == details::ContainerType::SMALL)
{
for (const auto & x : rhs.small)
insert(x);
}
else if (rhs.container_type == details::ContainerType::MEDIUM)
{
for (const auto & x : *rhs.medium)
insert(x);
}
}
else if (container_type == details::ContainerType::LARGE)
{
if (rhs.container_type == details::ContainerType::SMALL)
{
for (const auto & x : rhs.small)
insert(x);
}
else if (rhs.container_type == details::ContainerType::MEDIUM)
{
for (const auto & x : *rhs.medium)
insert(x);
}
else if (rhs.container_type == details::ContainerType::LARGE)
large->merge(*rhs.large);
}
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
}
/// Можно вызывать только для пустого объекта.
void read(DB::ReadBuffer & in)
{
UInt8 v;
readBinary(v, in);
details::ContainerType t = static_cast<details::ContainerType>(v);
if (t == details::ContainerType::SMALL)
small.read(in);
else if (t == details::ContainerType::MEDIUM)
{
toMedium();
medium->read(in);
}
else if (t == details::ContainerType::LARGE)
{
toLarge();
large->read(in);
}
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
}
void readAndMerge(DB::ReadBuffer & in)
{
Self other;
other.read(in);
merge(other);
}
void write(DB::WriteBuffer & out) const
{
UInt8 v = static_cast<UInt8>(container_type);
writeBinary(v, out);
if (container_type == details::ContainerType::SMALL)
small.write(out);
else if (container_type == details::ContainerType::MEDIUM)
medium->write(out);
else if (container_type == details::ContainerType::LARGE)
large->write(out);
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
}
bool isMedium() const
{
return container_type == details::ContainerType::MEDIUM;
}
private:
void toMedium()
{
if (container_type != details::ContainerType::SMALL)
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
if (current_memory_tracker)
current_memory_tracker->alloc(sizeof(medium));
Medium * tmp_medium = new Medium;
for (const auto & x : small)
tmp_medium->insert(x);
medium = tmp_medium;
container_type = details::ContainerType::MEDIUM;
}
void toLarge()
{
if ((container_type != details::ContainerType::SMALL) && (container_type != details::ContainerType::MEDIUM))
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
if (current_memory_tracker)
current_memory_tracker->alloc(sizeof(large));
Large * tmp_large = new Large;
if (container_type == details::ContainerType::SMALL)
{
for (const auto & x : small)
tmp_large->insert(x);
}
else if (container_type == details::ContainerType::MEDIUM)
{
for (const auto & x : *medium)
tmp_large->insert(x);
}
large = tmp_large;
if (container_type == details::ContainerType::MEDIUM)
{
delete medium;
medium = nullptr;
if (current_memory_tracker)
current_memory_tracker->free(sizeof(medium));
}
container_type = details::ContainerType::LARGE;
}
private:
Small small;
Medium * medium = nullptr;
Large * large = nullptr;
const UInt32 medium_set_size_max = 1UL << medium_set_power2_max;
details::ContainerType container_type = details::ContainerType::SMALL;
};
}

View File

@ -47,7 +47,7 @@ public:
for (size_t i = 0; i < structure.size(); ++i) for (size_t i = 0; i < structure.size(); ++i)
{ {
ColumnWithNameAndType column; ColumnWithTypeAndName column;
column.name = structure[i].first; column.name = structure[i].first;
column.type = data_type_factory.get(structure[i].second); column.type = data_type_factory.get(structure[i].second);
column.column = column.type->createColumn(); column.column = column.type->createColumn();

View File

@ -757,7 +757,7 @@ public:
{ {
Cell x; Cell x;
x.read(rb); x.read(rb);
insert(x); insert(Cell::getKey(x.getValue()));
} }
} }
@ -781,7 +781,7 @@ public:
Cell x; Cell x;
DB::assertString(",", rb); DB::assertString(",", rb);
x.readText(rb); x.readText(rb);
insert(x); insert(Cell::getKey(x.getValue()));
} }
} }

View File

@ -37,7 +37,7 @@ template<typename T1>
std::multiset<T1> extractSingleValueFromBlock(const Block & block, const String & name) std::multiset<T1> extractSingleValueFromBlock(const Block & block, const String & name)
{ {
std::multiset<T1> res; std::multiset<T1> res;
const ColumnWithNameAndType & data = block.getByName(name); const ColumnWithTypeAndName & data = block.getByName(name);
size_t rows = block.rows(); size_t rows = block.rows();
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
res.insert((*data.column)[i].get<T1>()); res.insert((*data.column)[i].get<T1>());

View File

@ -6,11 +6,11 @@
#include <initializer_list> #include <initializer_list>
#include <DB/Core/BlockInfo.h> #include <DB/Core/BlockInfo.h>
#include <DB/Core/ColumnWithNameAndType.h> #include <DB/Core/ColumnWithTypeAndName.h>
#include <DB/Core/NamesAndTypes.h> #include <DB/Core/NamesAndTypes.h>
#include <DB/Core/Exception.h> #include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include "ColumnsWithNameAndType.h" #include "ColumnsWithTypeAndName.h"
namespace DB namespace DB
@ -26,7 +26,7 @@ class Context;
class Block class Block
{ {
public: public:
typedef std::list<ColumnWithNameAndType> Container_t; typedef std::list<ColumnWithTypeAndName> Container_t;
typedef std::vector<Container_t::iterator> IndexByPosition_t; typedef std::vector<Container_t::iterator> IndexByPosition_t;
typedef std::map<String, Container_t::iterator> IndexByName_t; typedef std::map<String, Container_t::iterator> IndexByName_t;
@ -39,7 +39,7 @@ public:
BlockInfo info; BlockInfo info;
Block() = default; Block() = default;
Block(std::initializer_list<ColumnWithNameAndType> il) : data{il} Block(std::initializer_list<ColumnWithTypeAndName> il) : data{il}
{ {
index_by_position.reserve(il.size()); index_by_position.reserve(il.size());
for (auto it = std::begin(data); it != std::end(data); ++it) for (auto it = std::begin(data); it != std::end(data); ++it)
@ -56,11 +56,11 @@ public:
Block & operator= (Block && other) = default; Block & operator= (Block && other) = default;
/// вставить столбец в заданную позицию /// вставить столбец в заданную позицию
void insert(size_t position, const ColumnWithNameAndType & elem); void insert(size_t position, const ColumnWithTypeAndName & elem);
/// вставить столбец в конец /// вставить столбец в конец
void insert(const ColumnWithNameAndType & elem); void insert(const ColumnWithTypeAndName & elem);
/// вставить столбец в конец, если столбца с таким именем ещё нет /// вставить столбец в конец, если столбца с таким именем ещё нет
void insertUnique(const ColumnWithNameAndType & elem); void insertUnique(const ColumnWithTypeAndName & elem);
/// удалить столбец в заданной позиции /// удалить столбец в заданной позиции
void erase(size_t position); void erase(size_t position);
/// удалить столбец с заданным именем /// удалить столбец с заданным именем
@ -68,20 +68,20 @@ public:
/// Добавляет в блок недостающие столбцы со значениями по-умолчанию /// Добавляет в блок недостающие столбцы со значениями по-умолчанию
void addDefaults(const NamesAndTypesList & required_columns); void addDefaults(const NamesAndTypesList & required_columns);
ColumnWithNameAndType & getByPosition(size_t position); ColumnWithTypeAndName & getByPosition(size_t position);
const ColumnWithNameAndType & getByPosition(size_t position) const; const ColumnWithTypeAndName & getByPosition(size_t position) const;
ColumnWithNameAndType & unsafeGetByPosition(size_t position) { return *index_by_position[position]; } ColumnWithTypeAndName & unsafeGetByPosition(size_t position) { return *index_by_position[position]; }
const ColumnWithNameAndType & unsafeGetByPosition(size_t position) const { return *index_by_position[position]; } const ColumnWithTypeAndName & unsafeGetByPosition(size_t position) const { return *index_by_position[position]; }
ColumnWithNameAndType & getByName(const std::string & name); ColumnWithTypeAndName & getByName(const std::string & name);
const ColumnWithNameAndType & getByName(const std::string & name) const; const ColumnWithTypeAndName & getByName(const std::string & name) const;
bool has(const std::string & name) const; bool has(const std::string & name) const;
size_t getPositionByName(const std::string & name) const; size_t getPositionByName(const std::string & name) const;
ColumnsWithNameAndType getColumns() const; ColumnsWithTypeAndName getColumns() const;
NamesAndTypesList getColumnsList() const; NamesAndTypesList getColumnsList() const;
/** Возвращает количество строк в блоке. /** Возвращает количество строк в блоке.

View File

@ -14,19 +14,19 @@ using Poco::SharedPtr;
/** Тип данных для представления столбца вместе с его типом и именем в оперативке. /** Тип данных для представления столбца вместе с его типом и именем в оперативке.
*/ */
struct ColumnWithNameAndType struct ColumnWithTypeAndName
{ {
ColumnPtr column; ColumnPtr column;
DataTypePtr type; DataTypePtr type;
String name; String name;
ColumnWithNameAndType() {} ColumnWithTypeAndName() {}
ColumnWithNameAndType(const ColumnPtr & column_, const DataTypePtr & type_, const String name_) ColumnWithTypeAndName(const ColumnPtr & column_, const DataTypePtr & type_, const String name_)
: column(column_), type(type_), name(name_) {} : column(column_), type(type_), name(name_) {}
ColumnWithNameAndType cloneEmpty() const ColumnWithTypeAndName cloneEmpty() const
{ {
ColumnWithNameAndType res; ColumnWithTypeAndName res;
res.name = name; res.name = name;
res.type = type->clone(); res.type = type->clone();

View File

@ -1,13 +0,0 @@
#pragma once
#include <vector>
#include <DB/Core/ColumnWithNameAndType.h>
namespace DB
{
typedef std::vector<ColumnWithNameAndType> ColumnsWithNameAndType;
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <vector>
#include <DB/Core/ColumnWithTypeAndName.h>
namespace DB
{
typedef std::vector<ColumnWithTypeAndName> ColumnsWithTypeAndName;
}

View File

@ -180,7 +180,6 @@ namespace ErrorCodes
CANNOT_CREATE_DIRECTORY = 172, CANNOT_CREATE_DIRECTORY = 172,
CANNOT_ALLOCATE_MEMORY = 173, CANNOT_ALLOCATE_MEMORY = 173,
CYCLIC_ALIASES = 174, CYCLIC_ALIASES = 174,
NEGATIVE_REFCOUNT = 175,
CHUNK_NOT_FOUND = 176, CHUNK_NOT_FOUND = 176,
DUPLICATE_CHUNK_NAME = 177, DUPLICATE_CHUNK_NAME = 177,
MULTIPLE_ALIASES_FOR_EXPRESSION = 178, MULTIPLE_ALIASES_FOR_EXPRESSION = 178,

View File

@ -43,14 +43,31 @@ inline bool compareSSE2(const char * p1, const char * p2)
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p2)))); _mm_loadu_si128(reinterpret_cast<const __m128i *>(p2))));
} }
inline bool compareSSE2x4(const char * p1, const char * p2)
{
return 0xFFFF == _mm_movemask_epi8(
_mm_and_si128(
_mm_and_si128(
_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p1)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p2))),
_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p1) + 1),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p2) + 1))),
_mm_and_si128(
_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p1) + 2),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p2) + 2)),
_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p1) + 3),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p2) + 3)))));
}
inline bool memequalSSE2Wide(const char * p1, const char * p2, size_t size) inline bool memequalSSE2Wide(const char * p1, const char * p2, size_t size)
{ {
while (size >= 64) while (size >= 64)
{ {
if ( compareSSE2(p1, p2) if (compareSSE2x4(p1, p2))
&& compareSSE2(p1 + 16, p2 + 16)
&& compareSSE2(p1 + 32, p2 + 32)
&& compareSSE2(p1 + 48, p2 + 48))
{ {
p1 += 64; p1 += 64;
p2 += 64; p2 += 64;

View File

@ -4,7 +4,7 @@
#include <DB/DataStreams/IProfilingBlockInputStream.h> #include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Columns/ColumnConst.h> #include <DB/Columns/ColumnConst.h>
#include <DB/Core/ColumnWithNameAndType.h> #include <DB/Core/ColumnWithTypeAndName.h>
namespace DB namespace DB
{ {
@ -41,7 +41,7 @@ protected:
if (!res) if (!res)
return res; return res;
ColumnPtr column_ptr = ColumnConst<ColumnType>(res.rows(), value, data_type).convertToFullColumn(); ColumnPtr column_ptr = ColumnConst<ColumnType>(res.rows(), value, data_type).convertToFullColumn();
ColumnWithNameAndType column(column_ptr, data_type, column_name); ColumnWithTypeAndName column(column_ptr, data_type, column_name);
res.insert(column); res.insert(column);
return res; return res;
} }

View File

@ -25,14 +25,14 @@ struct BlockIO
Block out_sample; /// Пример блока, которого нужно писать в out. Block out_sample; /// Пример блока, которого нужно писать в out.
/// Здесь могут быть установлены колбэки для логгирования запроса. /// Здесь могут быть установлены колбэки для логгирования запроса.
std::function<void(IBlockInputStream &)> finish_callback; std::function<void(IBlockInputStream *)> finish_callback;
std::function<void()> exception_callback; std::function<void()> exception_callback;
/// Вызывайте эти функции, если нужно логгировать запрос. /// Вызывайте эти функции, если нужно логгировать запрос.
void onFinish() void onFinish()
{ {
if (in && finish_callback) if (finish_callback)
finish_callback(*in); finish_callback(in.get());
} }
void onException() void onException()

View File

@ -21,17 +21,21 @@ class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream
public: public:
/** Столбцы из key_names и аргументы агрегатных функций, уже должны быть вычислены. /** Столбцы из key_names и аргументы агрегатных функций, уже должны быть вычислены.
*/ */
ParallelAggregatingBlockInputStream(BlockInputStreams inputs, const Names & key_names, ParallelAggregatingBlockInputStream(
const AggregateDescriptions & aggregates, bool overflow_row_, bool final_, size_t max_threads_, BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end,
const Names & key_names, const AggregateDescriptions & aggregates,
bool overflow_row_, bool final_, size_t max_threads_,
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_,
Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_) Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_)
: aggregator(key_names, aggregates, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_, : aggregator(key_names, aggregates, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_,
compiler_, min_count_to_compile_, group_by_two_level_threshold_), compiler_, min_count_to_compile_, group_by_two_level_threshold_),
final(final_), max_threads(std::min(inputs.size(), max_threads_)), final(final_), max_threads(std::min(inputs.size(), max_threads_)),
keys_size(aggregator.getNumberOfKeys()), aggregates_size(aggregator.getNumberOfAggregates()), keys_size(aggregator.getNumberOfKeys()), aggregates_size(aggregator.getNumberOfAggregates()),
handler(*this), processor(inputs, max_threads, handler) handler(*this), processor(inputs, additional_input_at_end, max_threads, handler)
{ {
children.insert(children.end(), inputs.begin(), inputs.end()); children = inputs;
if (additional_input_at_end)
children.push_back(additional_input_at_end);
} }
String getName() const override { return "ParallelAggregating"; } String getName() const override { return "ParallelAggregating"; }

View File

@ -45,8 +45,16 @@ template <typename Handler>
class ParallelInputsProcessor class ParallelInputsProcessor
{ {
public: public:
ParallelInputsProcessor(BlockInputStreams inputs_, size_t max_threads_, Handler & handler_) /** additional_input_at_end - если не nullptr,
: inputs(inputs_), max_threads(std::min(inputs_.size(), max_threads_)), handler(handler_) * то из этого источника начинают доставаться блоки лишь после того, как все остальные источники обработаны.
* Это делается в основном потоке.
*
* Предназначено для реализации FULL и RIGHT JOIN
* - где нужно сначала параллельно сделать JOIN, при этом отмечая, какие ключи не найдены,
* и только после завершения этой работы, создать блоки из ненайденных ключей.
*/
ParallelInputsProcessor(BlockInputStreams inputs_, BlockInputStreamPtr additional_input_at_end_, size_t max_threads_, Handler & handler_)
: inputs(inputs_), additional_input_at_end(additional_input_at_end_), 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)
available_inputs.emplace(inputs_[i], i); available_inputs.emplace(inputs_[i], i);
@ -150,6 +158,25 @@ private:
/// Последний поток при выходе сообщает, что данных больше нет. /// Последний поток при выходе сообщает, что данных больше нет.
if (0 == --active_threads) if (0 == --active_threads)
{ {
/// И ещё обрабатывает дополнительный источник, если такой есть.
if (additional_input_at_end)
{
try
{
while (Block block = additional_input_at_end->read())
handler.onBlock(block, thread_num);
}
catch (...)
{
exception = cloneCurrentException();
}
if (exception)
{
handler.onException(exception, thread_num);
}
}
handler.onFinish(); handler.onFinish();
} }
} }
@ -206,6 +233,7 @@ private:
} }
BlockInputStreams inputs; BlockInputStreams inputs;
BlockInputStreamPtr additional_input_at_end;
unsigned max_threads; unsigned max_threads;
Handler & handler; Handler & handler;

View File

@ -0,0 +1,24 @@
#pragma once
#include <DB/DataStreams/TabSeparatedRowOutputStream.h>
namespace DB
{
/** Поток для вывода данных в формате TSKV.
* TSKV похож на TabSeparated, но перед каждым значением указывается его имя и знак равенства: name=value.
* Этот формат весьма неэффективен.
*/
class TSKVRowOutputStream : public TabSeparatedRowOutputStream
{
public:
TSKVRowOutputStream(WriteBuffer & ostr_, const Block & sample_);
void writeField(const Field & field) override;
protected:
NamesAndTypes fields;
};
}

View File

@ -28,12 +28,14 @@ using Poco::SharedPtr;
class UnionBlockInputStream : public IProfilingBlockInputStream class UnionBlockInputStream : public IProfilingBlockInputStream
{ {
public: public:
UnionBlockInputStream(BlockInputStreams inputs, size_t max_threads) : UnionBlockInputStream(BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end, size_t max_threads) :
output_queue(std::min(inputs.size(), max_threads)), output_queue(std::min(inputs.size(), max_threads)),
handler(*this), handler(*this),
processor(inputs, max_threads, handler) processor(inputs, additional_input_at_end, max_threads, handler)
{ {
children = inputs; children = inputs;
if (additional_input_at_end)
children.push_back(additional_input_at_end);
} }
String getName() const override { return "Union"; } String getName() const override { return "Union"; }

View File

@ -136,7 +136,7 @@ public:
Block tuple_block; Block tuple_block;
for (size_t i = 0, size = elems.size(); i < size; ++i) for (size_t i = 0, size = elems.size(); i < size; ++i)
{ {
ColumnWithNameAndType col; ColumnWithTypeAndName col;
col.column = elems[i]->createColumn(); col.column = elems[i]->createColumn();
col.type = elems[i]->clone(); col.type = elems[i]->clone();
tuple_block.insert(col); tuple_block.insert(col);

View File

@ -27,7 +27,7 @@ public:
void serializeText(const Field & field, WriteBuffer & ostr) const void serializeText(const Field & field, WriteBuffer & ostr) const
{ {
writeText(get<typename NearestFieldType<FieldType>::Type>(field), ostr); writeText(static_cast<FieldType>(get<typename NearestFieldType<FieldType>::Type>(field)), ostr);
} }
inline void deserializeText(Field & field, ReadBuffer & istr) const; inline void deserializeText(Field & field, ReadBuffer & istr) const;

View File

@ -87,61 +87,11 @@ public:
bool hasHierarchy() const override { return hierarchical_attribute; } bool hasHierarchy() const override { return hierarchical_attribute; }
id_t toParent(const id_t id) const override
{
PODArray<UInt64> ids{1, id};
PODArray<UInt64> out{1};
getItems<UInt64>(*hierarchical_attribute, ids, out);
return out.front();
}
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
{ {
getItems<UInt64>(*hierarchical_attribute, ids, out); getItems<UInt64>(*hierarchical_attribute, ids, out);
} }
#define DECLARE_INDIVIDUAL_GETTER(TYPE) \
TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\
{\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
PODArray<UInt64> ids{1, id};\
PODArray<TYPE> out{1};\
getItems<TYPE>(attribute, ids, out);\
return out.front();\
}
DECLARE_INDIVIDUAL_GETTER(UInt8)
DECLARE_INDIVIDUAL_GETTER(UInt16)
DECLARE_INDIVIDUAL_GETTER(UInt32)
DECLARE_INDIVIDUAL_GETTER(UInt64)
DECLARE_INDIVIDUAL_GETTER(Int8)
DECLARE_INDIVIDUAL_GETTER(Int16)
DECLARE_INDIVIDUAL_GETTER(Int32)
DECLARE_INDIVIDUAL_GETTER(Int64)
DECLARE_INDIVIDUAL_GETTER(Float32)
DECLARE_INDIVIDUAL_GETTER(Float64)
#undef DECLARE_INDIVIDUAL_GETTER
String getString(const std::string & attribute_name, const id_t id) const override
{
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
PODArray<UInt64> ids{1, id};
ColumnString out;
getItems(attribute, ids, &out);
return String{out.getDataAt(0)};
};
#define DECLARE_MULTIPLE_GETTER(TYPE)\ #define DECLARE_MULTIPLE_GETTER(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\
{\ {\

View File

@ -95,6 +95,14 @@ private:
writeProbablyBackQuotedString(dict_struct.id_name, out); writeProbablyBackQuotedString(dict_struct.id_name, out);
if (!dict_struct.range_min.empty() && !dict_struct.range_max.empty())
{
writeString(", ", out);
writeProbablyBackQuotedString(dict_struct.range_min, out);
writeString(", ", out);
writeProbablyBackQuotedString(dict_struct.range_max, out);
}
for (const auto & attr : dict_struct.attributes) for (const auto & attr : dict_struct.attributes)
{ {
writeString(", ", out); writeString(", ", out);

View File

@ -18,7 +18,7 @@ namespace
Block createSampleBlock(const DictionaryStructure & dict_struct) Block createSampleBlock(const DictionaryStructure & dict_struct)
{ {
Block block{ Block block{
ColumnWithNameAndType{ ColumnWithTypeAndName{
new ColumnUInt64, new ColumnUInt64,
new DataTypeUInt64, new DataTypeUInt64,
dict_struct.id_name dict_struct.id_name
@ -26,7 +26,7 @@ Block createSampleBlock(const DictionaryStructure & dict_struct)
}; };
for (const auto & attribute : dict_struct.attributes) for (const auto & attribute : dict_struct.attributes)
block.insert(ColumnWithNameAndType{ block.insert(ColumnWithTypeAndName{
attribute.type->createColumn(), attribute.type, attribute.name attribute.type->createColumn(), attribute.type, attribute.name
}); });

View File

@ -117,9 +117,13 @@ struct DictionaryStructure final
{ {
std::string id_name; std::string id_name;
std::vector<DictionaryAttribute> attributes; std::vector<DictionaryAttribute> attributes;
std::string range_min;
std::string range_max;
DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
: id_name{config.getString(config_prefix + ".id.name")} : id_name{config.getString(config_prefix + ".id.name")},
range_min{config.getString(config_prefix + ".range_min.name", "")},
range_max{config.getString(config_prefix + ".range_max.name", "")}
{ {
if (id_name.empty()) if (id_name.empty())
throw Exception{ throw Exception{

View File

@ -81,64 +81,11 @@ public:
bool hasHierarchy() const override { return hierarchical_attribute; } bool hasHierarchy() const override { return hierarchical_attribute; }
id_t toParent(const id_t id) const override
{
const auto attr = hierarchical_attribute;
const auto & array = *std::get<std::unique_ptr<PODArray<UInt64>>>(attr->arrays);
query_count.fetch_add(1, std::memory_order_relaxed);
return id < array.size() ? array[id] : std::get<UInt64>(attr->null_values);
}
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
{ {
getItems<UInt64>(*hierarchical_attribute, ids, out); getItems<UInt64>(*hierarchical_attribute, ids, out);
} }
#define DECLARE_INDIVIDUAL_GETTER(TYPE) \
TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
const auto & array = *std::get<std::unique_ptr<PODArray<TYPE>>>(attribute.arrays);\
\
query_count.fetch_add(1, std::memory_order_relaxed);\
\
return id < array.size() ? array[id] : std::get<TYPE>(attribute.null_values);\
}
DECLARE_INDIVIDUAL_GETTER(UInt8)
DECLARE_INDIVIDUAL_GETTER(UInt16)
DECLARE_INDIVIDUAL_GETTER(UInt32)
DECLARE_INDIVIDUAL_GETTER(UInt64)
DECLARE_INDIVIDUAL_GETTER(Int8)
DECLARE_INDIVIDUAL_GETTER(Int16)
DECLARE_INDIVIDUAL_GETTER(Int32)
DECLARE_INDIVIDUAL_GETTER(Int64)
DECLARE_INDIVIDUAL_GETTER(Float32)
DECLARE_INDIVIDUAL_GETTER(Float64)
#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 != AttributeUnderlyingType::String)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
const auto & array = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays);
query_count.fetch_add(1, std::memory_order_relaxed);
return id < array.size() ? String{array[id]} : std::get<String>(attribute.null_values);
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\ #define DECLARE_MULTIPLE_GETTER(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\
{\ {\

View File

@ -10,6 +10,7 @@
#include <memory> #include <memory>
#include <tuple> #include <tuple>
namespace DB namespace DB
{ {
@ -78,67 +79,11 @@ public:
bool hasHierarchy() const override { return hierarchical_attribute; } bool hasHierarchy() const override { return hierarchical_attribute; }
id_t toParent(const id_t id) const override
{
const auto attr = hierarchical_attribute;
const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, UInt64>>>(attr->maps);
const auto it = map.find(id);
query_count.fetch_add(1, std::memory_order_relaxed);
return it != map.end() ? it->second : std::get<UInt64>(attr->null_values);
}
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
{ {
getItems<UInt64>(*hierarchical_attribute, ids, out); getItems<UInt64>(*hierarchical_attribute, ids, out);
} }
#define DECLARE_INDIVIDUAL_GETTER(TYPE) \
TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
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, TYPE>>>(attribute.maps);\
const auto it = map.find(id);\
\
query_count.fetch_add(1, std::memory_order_relaxed);\
\
return it != map.end() ? TYPE{it->second} : std::get<TYPE>(attribute.null_values);\
}
DECLARE_INDIVIDUAL_GETTER(UInt8)
DECLARE_INDIVIDUAL_GETTER(UInt16)
DECLARE_INDIVIDUAL_GETTER(UInt32)
DECLARE_INDIVIDUAL_GETTER(UInt64)
DECLARE_INDIVIDUAL_GETTER(Int8)
DECLARE_INDIVIDUAL_GETTER(Int16)
DECLARE_INDIVIDUAL_GETTER(Int32)
DECLARE_INDIVIDUAL_GETTER(Int64)
DECLARE_INDIVIDUAL_GETTER(Float32)
DECLARE_INDIVIDUAL_GETTER(Float64)
#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 != AttributeUnderlyingType::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);
query_count.fetch_add(1, std::memory_order_relaxed);
return it != map.end() ? String{it->second} : std::get<String>(attribute.null_values);
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\ #define DECLARE_MULTIPLE_GETTER(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\
{\ {\

View File

@ -12,16 +12,15 @@ namespace DB
class IDictionarySource; class IDictionarySource;
class IDictionary; class IDictionaryBase;
using DictionaryPtr = std::unique_ptr<IDictionary>; using DictionaryPtr = std::unique_ptr<IDictionaryBase>;
class DictionaryLifetime; class DictionaryLifetime;
class DictionaryStructure; class DictionaryStructure;
class ColumnString; class ColumnString;
class IDictionary struct IDictionaryBase
{ {
public:
using id_t = std::uint64_t; using id_t = std::uint64_t;
virtual std::exception_ptr getCreationException() const = 0; virtual std::exception_ptr getCreationException() const = 0;
@ -53,10 +52,24 @@ public:
virtual bool isInjective(const std::string & attribute_name) const = 0; virtual bool isInjective(const std::string & attribute_name) const = 0;
virtual ~IDictionaryBase() = default;
};
struct IDictionary : IDictionaryBase
{
virtual bool hasHierarchy() const = 0; virtual bool hasHierarchy() const = 0;
/// do not call unless you ensure that hasHierarchy() returns true /// do not call unless you ensure that hasHierarchy() returns true
virtual id_t toParent(id_t id) const = 0; id_t toParent(id_t id) const
{
const PODArray<UInt64> ids(1, id);
PODArray<UInt64> out(1);
toParent(ids, out);
return out.front();
}
virtual void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const = 0; virtual void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const = 0;
bool in(id_t child_id, const id_t ancestor_id) const bool in(id_t child_id, const id_t ancestor_id) const
@ -67,20 +80,7 @@ public:
return child_id != 0; return child_id != 0;
} }
/// functions for individual access /// return mapped values for a collection of identifiers
virtual UInt8 getUInt8(const std::string & attribute_name, id_t id) const = 0;
virtual UInt16 getUInt16(const std::string & attribute_name, id_t id) const = 0;
virtual UInt32 getUInt32(const std::string & attribute_name, id_t id) const = 0;
virtual UInt64 getUInt64(const std::string & attribute_name, id_t id) const = 0;
virtual Int8 getInt8(const std::string & attribute_name, id_t id) const = 0;
virtual Int16 getInt16(const std::string & attribute_name, id_t id) const = 0;
virtual Int32 getInt32(const std::string & attribute_name, id_t id) const = 0;
virtual Int64 getInt64(const std::string & attribute_name, id_t id) const = 0;
virtual Float32 getFloat32(const std::string & attribute_name, id_t id) const = 0;
virtual Float64 getFloat64(const std::string & attribute_name, id_t id) const = 0;
virtual String getString(const std::string & attribute_name, id_t id) const = 0;
/// functions for multiple access
virtual void getUInt8(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt8> & out) const = 0; virtual void getUInt8(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt8> & out) const = 0;
virtual void getUInt16(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt16> & out) const = 0; virtual void getUInt16(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt16> & out) const = 0;
virtual void getUInt32(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt32> & out) const = 0; virtual void getUInt32(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt32> & out) const = 0;
@ -92,8 +92,6 @@ public:
virtual void getFloat32(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Float32> & out) const = 0; virtual void getFloat32(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Float32> & out) const = 0;
virtual void getFloat64(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Float64> & out) const = 0; virtual void getFloat64(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Float64> & out) const = 0;
virtual void getString(const std::string & attr_name, const PODArray<id_t> & ids, ColumnString * out) const = 0; virtual void getString(const std::string & attr_name, const PODArray<id_t> & ids, ColumnString * out) const = 0;
virtual ~IDictionary() = default;
}; };
} }

View File

@ -106,6 +106,14 @@ private:
writeProbablyBackQuotedString(dict_struct.id_name, out); writeProbablyBackQuotedString(dict_struct.id_name, out);
if (!dict_struct.range_min.empty() && !dict_struct.range_max.empty())
{
writeString(", ", out);
writeProbablyBackQuotedString(dict_struct.range_min, out);
writeString(", ", out);
writeProbablyBackQuotedString(dict_struct.range_max, out);
}
for (const auto & attr : dict_struct.attributes) for (const auto & attr : dict_struct.attributes)
{ {
writeString(", ", out); writeString(", ", out);

View File

@ -0,0 +1,427 @@
#pragma once
#include <DB/Dictionaries/IDictionary.h>
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp>
#include <atomic>
#include <memory>
#include <tuple>
namespace DB
{
class RangeHashedDictionary final : public IDictionaryBase
{
public:
RangeHashedDictionary(
const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime)
: name{name}, dict_struct(dict_struct),
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime)
{
createAttributes();
try
{
loadData();
calculateBytesAllocated();
}
catch (...)
{
creation_exception = std::current_exception();
}
creation_time = std::chrono::system_clock::now();
}
RangeHashedDictionary(const RangeHashedDictionary & other)
: RangeHashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime}
{}
std::exception_ptr getCreationException() const override { return creation_exception; }
std::string getName() const override { return name; }
std::string getTypeName() const override { return "RangeHashed"; }
std::size_t getBytesAllocated() const override { return bytes_allocated; }
std::size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
double getHitRate() const override { return 1.0; }
std::size_t getElementCount() const override { return element_count; }
double getLoadFactor() const override { return static_cast<double>(element_count) / bucket_count; }
bool isCached() const override { return false; }
DictionaryPtr clone() const override { return std::make_unique<RangeHashedDictionary>(*this); }
const IDictionarySource * getSource() const override { return source_ptr.get(); }
const DictionaryLifetime & getLifetime() const override { return dict_lifetime; }
const DictionaryStructure & getStructure() const override { return dict_struct; }
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
bool isInjective(const std::string & attribute_name) const override
{
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const PODArray<UInt16> & dates,\
PODArray<TYPE> & out) const\
{\
const auto & attribute = getAttributeWithType(attribute_name, AttributeUnderlyingType::TYPE);\
getItems<TYPE>(attribute, ids, dates, out);\
}
DECLARE_MULTIPLE_GETTER(UInt8)
DECLARE_MULTIPLE_GETTER(UInt16)
DECLARE_MULTIPLE_GETTER(UInt32)
DECLARE_MULTIPLE_GETTER(UInt64)
DECLARE_MULTIPLE_GETTER(Int8)
DECLARE_MULTIPLE_GETTER(Int16)
DECLARE_MULTIPLE_GETTER(Int32)
DECLARE_MULTIPLE_GETTER(Int64)
DECLARE_MULTIPLE_GETTER(Float32)
DECLARE_MULTIPLE_GETTER(Float64)
#undef DECLARE_MULTIPLE_GETTER
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const PODArray<UInt16> & dates,
ColumnString * out) const
{
const auto & attribute = getAttributeWithType(attribute_name, AttributeUnderlyingType::String);
const auto & attr = *std::get<ptr_t<StringRef>>(attribute.maps);
const auto & null_value = std::get<String>(attribute.null_values);
for (const auto i : ext::range(0, ids.size()))
{
const auto it = attr.find(ids[i]);
if (it != std::end(attr))
{
const auto date = dates[i];
const auto & ranges_and_values = it->second;
const auto val_it = std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values),
[date] (const value_t<StringRef> & v) { return v.range.contains(date); });
const auto string_ref = val_it != std::end(ranges_and_values) ? val_it->value : null_value;
out->insertData(string_ref.data, string_ref.size);
}
else
out->insertData(null_value.data(), null_value.size());
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
}
private:
struct range_t : std::pair<UInt16, UInt16>
{
using std::pair<UInt16, UInt16>::pair;
bool contains(const UInt16 date) const
{
const auto & left = first;
const auto & right = second;
if (left <= date && date <= right)
return true;
const auto has_left_bound = 0 < left && left <= DATE_LUT_MAX_DAY_NUM;
const auto has_right_bound = 0 < right && right <= DATE_LUT_MAX_DAY_NUM;
if ((!has_left_bound || left <= date) && (!has_right_bound || date <= right))
return true;
return false;
}
};
template <typename T>
struct value_t final
{
range_t range;
T value;
};
template <typename T> using values_t = std::vector<value_t<T>>;
template <typename T> using collection_t = HashMap<UInt64, values_t<T>>;
template <typename T> using ptr_t = std::unique_ptr<collection_t<T>>;
struct attribute_t final
{
public:
AttributeUnderlyingType type;
std::tuple<UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
String> null_values;
std::tuple<ptr_t<UInt8>, ptr_t<UInt16>, ptr_t<UInt32>, ptr_t<UInt64>,
ptr_t<Int8>, ptr_t<Int16>, ptr_t<Int32>, ptr_t<Int64>,
ptr_t<Float32>, ptr_t<Float64>, ptr_t<StringRef>> maps;
std::unique_ptr<Arena> string_arena;
};
void createAttributes()
{
const auto size = dict_struct.attributes.size();
attributes.reserve(size);
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
throw Exception{
"Hierarchical attributes not supported by " + getName() + " dictionary.",
ErrorCodes::BAD_ARGUMENTS
};
}
}
void loadData()
{
auto stream = source_ptr->loadAll();
stream->readPrefix();
while (const auto block = stream->read())
{
const auto & id_column = *block.getByPosition(0).column;
const auto & min_range_column = *block.getByPosition(1).column;
const auto & max_range_column = *block.getByPosition(2).column;
element_count += id_column.size();
for (const auto attribute_idx : ext::range(0, attributes.size()))
{
const auto & attribute_column = *block.getByPosition(attribute_idx + 3).column;
auto & attribute = attributes[attribute_idx];
for (const auto row_idx : ext::range(0, id_column.size()))
setAttributeValue(attribute, id_column[row_idx].get<UInt64>(),
range_t(min_range_column[row_idx].get<UInt64>(), max_range_column[row_idx].get<UInt64>()),
attribute_column[row_idx]);
}
}
stream->readSuffix();
}
template <typename T>
void addAttributeSize(const attribute_t & attribute)
{
const auto & map_ref = std::get<ptr_t<T>>(attribute.maps);
bytes_allocated += sizeof(collection_t<T>) + map_ref->getBufferSizeInBytes();
bucket_count = map_ref->getBufferSizeInCells();
}
void calculateBytesAllocated()
{
bytes_allocated += attributes.size() * sizeof(attributes.front());
for (const auto & attribute : attributes)
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: addAttributeSize<UInt8>(attribute); break;
case AttributeUnderlyingType::UInt16: addAttributeSize<UInt16>(attribute); break;
case AttributeUnderlyingType::UInt32: addAttributeSize<UInt32>(attribute); break;
case AttributeUnderlyingType::UInt64: addAttributeSize<UInt64>(attribute); break;
case AttributeUnderlyingType::Int8: addAttributeSize<Int8>(attribute); break;
case AttributeUnderlyingType::Int16: addAttributeSize<Int16>(attribute); break;
case AttributeUnderlyingType::Int32: addAttributeSize<Int32>(attribute); break;
case AttributeUnderlyingType::Int64: addAttributeSize<Int64>(attribute); break;
case AttributeUnderlyingType::Float32: addAttributeSize<Float32>(attribute); break;
case AttributeUnderlyingType::Float64: addAttributeSize<Float64>(attribute); break;
case AttributeUnderlyingType::String:
{
addAttributeSize<StringRef>(attribute);
bytes_allocated += sizeof(Arena) + attribute.string_arena->size();
break;
}
}
}
}
template <typename T>
void createAttributeImpl(attribute_t & attribute, const Field & null_value)
{
std::get<T>(attribute.null_values) = null_value.get<typename NearestFieldType<T>::Type>();
std::get<ptr_t<T>>(attribute.maps) = std::make_unique<collection_t<T>>();
}
attribute_t createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
{
attribute_t attr{type};
switch (type)
{
case AttributeUnderlyingType::UInt8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeUnderlyingType::UInt16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeUnderlyingType::UInt32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeUnderlyingType::UInt64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeUnderlyingType::Int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeUnderlyingType::Int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeUnderlyingType::Int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeUnderlyingType::Int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeUnderlyingType::Float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeUnderlyingType::String:
{
const auto & null_value_ref = std::get<String>(attr.null_values) = null_value.get<String>();
std::get<ptr_t<StringRef>>(attr.maps) = std::make_unique<collection_t<StringRef>>();
attr.string_arena = std::make_unique<Arena>();
break;
}
}
return attr;
}
template <typename T>
void getItems(
const attribute_t & attribute, const PODArray<id_t> & ids, const PODArray<UInt16> & dates,
PODArray<T> & out) const
{
const auto & attr = *std::get<ptr_t<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]);
if (it != std::end(attr))
{
const auto date = dates[i];
const auto & ranges_and_values = it->second;
const auto val_it = std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values),
[date] (const value_t<T> & v) { return v.range.contains(date); });
out[i] = val_it != std::end(ranges_and_values) ? val_it->value : null_value;
}
else
out[i] = null_value;
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
}
template <typename T>
void setAttributeValueImpl(attribute_t & attribute, const id_t id, const range_t & range, const T value)
{
auto & map = *std::get<ptr_t<T>>(attribute.maps);
const auto it = map.find(id);
if (it != map.end())
{
auto & values = it->second;
const auto insert_it = std::lower_bound(std::begin(values), std::end(values), range,
[] (const value_t<T> & lhs, const range_t & range) {
return lhs.range < range;
});
values.insert(insert_it, value_t<T>{ range, value });
}
else
map.insert({ id, values_t<T>{ value_t<T>{ range, value } } });
}
void setAttributeValue(attribute_t & attribute, const id_t id, const range_t & range, const Field & value)
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: setAttributeValueImpl<UInt8>(attribute, id, range, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt16: setAttributeValueImpl<UInt16>(attribute, id, range, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt32: setAttributeValueImpl<UInt32>(attribute, id, range, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt64: setAttributeValueImpl<UInt64>(attribute, id, range, value.get<UInt64>()); break;
case AttributeUnderlyingType::Int8: setAttributeValueImpl<Int8>(attribute, id, range, value.get<Int64>()); break;
case AttributeUnderlyingType::Int16: setAttributeValueImpl<Int16>(attribute, id, range, value.get<Int64>()); break;
case AttributeUnderlyingType::Int32: setAttributeValueImpl<Int32>(attribute, id, range, value.get<Int64>()); break;
case AttributeUnderlyingType::Int64: setAttributeValueImpl<Int64>(attribute, id, range, value.get<Int64>()); break;
case AttributeUnderlyingType::Float32: setAttributeValueImpl<Float32>(attribute, id, range, value.get<Float64>()); break;
case AttributeUnderlyingType::Float64: setAttributeValueImpl<Float64>(attribute, id, range, value.get<Float64>()); break;
case AttributeUnderlyingType::String:
{
auto & map = *std::get<ptr_t<StringRef>>(attribute.maps);
const auto & string = value.get<String>();
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
const StringRef string_ref{string_in_arena, string.size()};
const auto it = map.find(id);
if (it != map.end())
{
auto & values = it->second;
const auto insert_it = std::lower_bound(std::begin(values), std::end(values), range,
[] (const value_t<StringRef> & lhs, const range_t & range) {
return lhs.range < range;
});
values.insert(insert_it, value_t<StringRef>{ range, string_ref });
}
else
map.insert({ id, values_t<StringRef>{ value_t<StringRef>{ range, string_ref } } });
break;
}
}
}
const attribute_t & getAttribute(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
throw Exception{
"No such attribute '" + attribute_name + "'",
ErrorCodes::BAD_ARGUMENTS
};
return attributes[it->second];
}
const attribute_t & getAttributeWithType(const std::string & name, const AttributeUnderlyingType type) const
{
const auto & attribute = getAttribute(name);
if (attribute.type != type)
throw Exception{
"Type mismatch: attribute " + name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
return attribute;
}
const std::string name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
std::map<std::string, std::size_t> attribute_index_by_name;
std::vector<attribute_t> attributes;
std::size_t bytes_allocated = 0;
std::size_t element_count = 0;
std::size_t bucket_count = 0;
mutable std::atomic<std::size_t> query_count{};
std::chrono::time_point<std::chrono::system_clock> creation_time;
std::exception_ptr creation_exception;
};
}

View File

@ -634,12 +634,12 @@ private:
for (size_t i = 0; i < tuple_size; ++i) for (size_t i = 0; i < tuple_size; ++i)
{ {
ColumnWithNameAndType array_of_tuple_section; ColumnWithTypeAndName array_of_tuple_section;
array_of_tuple_section.column = new ColumnArray(tuple_block.getByPosition(i).column, col_array->getOffsetsColumn()); array_of_tuple_section.column = new ColumnArray(tuple_block.getByPosition(i).column, col_array->getOffsetsColumn());
array_of_tuple_section.type = new DataTypeArray(tuple_block.getByPosition(i).type); array_of_tuple_section.type = new DataTypeArray(tuple_block.getByPosition(i).type);
block_of_temporary_results.insert(array_of_tuple_section); block_of_temporary_results.insert(array_of_tuple_section);
ColumnWithNameAndType array_elements_of_tuple_section; ColumnWithTypeAndName array_elements_of_tuple_section;
block_of_temporary_results.insert(array_elements_of_tuple_section); block_of_temporary_results.insert(array_elements_of_tuple_section);
execute(block_of_temporary_results, ColumnNumbers{i * 2 + 1, 0}, i * 2 + 2); execute(block_of_temporary_results, ColumnNumbers{i * 2 + 1, 0}, i * 2 + 2);
@ -1360,7 +1360,17 @@ private:
{ {
const auto & in_data = in->getData(); const auto & in_data = in->getData();
const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), std::size_t{}, const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), std::size_t{},
std::plus<std::size_t>{}); [this] (const std::size_t lhs, const std::size_t rhs) {
const auto sum = lhs + rhs;
if (sum < lhs)
throw Exception{
"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
ErrorCodes::ARGUMENT_OUT_OF_BOUND
};
return sum;
});
if (total_values > max_elements) if (total_values > max_elements)
throw Exception{ throw Exception{
"A call to function " + getName() + " would produce " + std::to_string(total_values) + "A call to function " + getName() + " would produce " + std::to_string(total_values) +
@ -1391,6 +1401,12 @@ private:
else if (const auto in = typeid_cast<const ColumnConst<T> *>(arg)) else if (const auto in = typeid_cast<const ColumnConst<T> *>(arg))
{ {
const auto & in_data = in->getData(); const auto & in_data = in->getData();
if (in->size() > std::numeric_limits<std::size_t>::max() / in_data)
throw Exception{
"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
ErrorCodes::ARGUMENT_OUT_OF_BOUND
};
const std::size_t total_values = in->size() * in_data; const std::size_t total_values = in->size() * in_data;
if (total_values > max_elements) if (total_values > max_elements)
throw Exception{ throw Exception{

View File

@ -2,10 +2,13 @@
#include <DB/DataTypes/DataTypesNumberFixed.h> #include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeArray.h> #include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/Columns/ColumnVector.h> #include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnConst.h> #include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnArray.h> #include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Functions/IFunction.h> #include <DB/Functions/IFunction.h>
#include <DB/Functions/NumberTraits.h> #include <DB/Functions/NumberTraits.h>
@ -170,7 +173,83 @@ struct StringIfImpl
} }
} }
static void vector_constant( static void vector_fixed_vector_fixed(
const PODArray<UInt8> & cond,
const ColumnFixedString::Chars_t & a_data,
const ColumnFixedString::Chars_t & b_data,
const size_t N,
ColumnFixedString::Chars_t & c_data)
{
size_t size = cond.size();
c_data.resize(a_data.size());
for (size_t i = 0; i < size; ++i)
{
if (cond[i])
memcpy(&c_data[i * N], &a_data[i * N], N);
else
memcpy(&c_data[i * N], &b_data[i * N], N);
}
}
template <bool negative>
static void vector_vector_fixed_impl(
const PODArray<UInt8> & cond,
const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets,
const ColumnFixedString::Chars_t & b_data, const size_t b_N,
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
{
size_t size = cond.size();
c_offsets.resize(size);
c_data.reserve(std::max(a_data.size(), b_data.size() + size));
ColumnString::Offset_t a_prev_offset = 0;
ColumnString::Offset_t c_prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
if (negative != cond[i])
{
size_t size_to_write = a_offsets[i] - a_prev_offset;
c_data.resize(c_data.size() + size_to_write);
memcpy(&c_data[c_prev_offset], &a_data[a_prev_offset], size_to_write);
c_prev_offset += size_to_write;
c_offsets[i] = c_prev_offset;
}
else
{
size_t size_to_write = b_N;
c_data.resize(c_data.size() + size_to_write + 1);
memcpy(&c_data[c_prev_offset], &b_data[i * b_N], size_to_write);
c_data.back() = 0;
c_prev_offset += size_to_write + 1;
c_offsets[i] = c_prev_offset;
}
a_prev_offset = a_offsets[i];
}
}
static void vector_vector_fixed(
const PODArray<UInt8> & cond,
const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets,
const ColumnFixedString::Chars_t & b_data, const size_t b_N,
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
{
vector_vector_fixed_impl<false>(cond, a_data, a_offsets, b_data, b_N, c_data, c_offsets);
}
static void vector_fixed_vector(
const PODArray<UInt8> & cond,
const ColumnFixedString::Chars_t & a_data, const size_t a_N,
const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets,
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
{
vector_vector_fixed_impl<true>(cond, b_data, b_offsets, a_data, a_N, c_data, c_offsets);
}
template <bool negative>
static void vector_constant_impl(
const PODArray<UInt8> & cond, const PODArray<UInt8> & cond,
const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets,
const String & b, const String & b,
@ -185,7 +264,7 @@ struct StringIfImpl
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
if (cond[i]) if (negative != cond[i])
{ {
size_t size_to_write = a_offsets[i] - a_prev_offset; size_t size_to_write = a_offsets[i] - a_prev_offset;
c_data.resize(c_data.size() + size_to_write); c_data.resize(c_data.size() + size_to_write);
@ -206,42 +285,77 @@ struct StringIfImpl
} }
} }
static void vector_constant(
const PODArray<UInt8> & cond,
const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets,
const String & b,
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
{
return vector_constant_impl<false>(cond, a_data, a_offsets, b, c_data, c_offsets);
}
static void constant_vector( static void constant_vector(
const PODArray<UInt8> & cond, const PODArray<UInt8> & cond,
const String & a, const String & a,
const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets,
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets) ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
{
return vector_constant_impl<true>(cond, b_data, b_offsets, a, c_data, c_offsets);
}
template <bool negative>
static void vector_fixed_constant_impl(
const PODArray<UInt8> & cond,
const ColumnFixedString::Chars_t & a_data, const size_t a_N,
const String & b,
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
{ {
size_t size = cond.size(); size_t size = cond.size();
c_offsets.resize(size); c_offsets.resize(size);
c_data.reserve(b_data.size()); c_data.reserve(a_data.size());
ColumnString::Offset_t b_prev_offset = 0;
ColumnString::Offset_t c_prev_offset = 0; ColumnString::Offset_t c_prev_offset = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
if (cond[i]) if (negative != cond[i])
{ {
size_t size_to_write = a.size() + 1; size_t size_to_write = a_N;
c_data.resize(c_data.size() + size_to_write); c_data.resize(c_data.size() + size_to_write + 1);
memcpy(&c_data[c_prev_offset], a.data(), size_to_write); memcpy(&c_data[c_prev_offset], &a_data[i * a_N], size_to_write);
c_prev_offset += size_to_write; c_data.back() = 0;
c_prev_offset += size_to_write + 1;
c_offsets[i] = c_prev_offset; c_offsets[i] = c_prev_offset;
} }
else else
{ {
size_t size_to_write = b_offsets[i] - b_prev_offset; size_t size_to_write = b.size() + 1;
c_data.resize(c_data.size() + size_to_write); c_data.resize(c_data.size() + size_to_write);
memcpy(&c_data[c_prev_offset], &b_data[b_prev_offset], size_to_write); memcpy(&c_data[c_prev_offset], b.data(), size_to_write);
c_prev_offset += size_to_write; c_prev_offset += size_to_write;
c_offsets[i] = c_prev_offset; c_offsets[i] = c_prev_offset;
} }
b_prev_offset = b_offsets[i];
} }
} }
static void vector_fixed_constant(
const PODArray<UInt8> & cond,
const ColumnFixedString::Chars_t & a_data, const size_t N,
const String & b,
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
{
vector_fixed_constant_impl<false>(cond, a_data, N, b, c_data, c_offsets);
}
static void constant_vector_fixed(
const PODArray<UInt8> & cond,
const String & a,
const ColumnFixedString::Chars_t & b_data, const size_t N,
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
{
vector_fixed_constant_impl<true>(cond, b_data, N, a, c_data, c_offsets);
}
static void constant_constant( static void constant_constant(
const PODArray<UInt8> & cond, const PODArray<UInt8> & cond,
const String & a, const String & b, const String & a, const String & b,
@ -495,6 +609,7 @@ public:
/** Реализация для массивов строк. /** Реализация для массивов строк.
* NOTE: Код слишком сложный, потому что он работает в внутренностями массивов строк. * NOTE: Код слишком сложный, потому что он работает в внутренностями массивов строк.
* NOTE: Массивы из FixedString не поддерживаются.
*/ */
struct StringArrayIfImpl struct StringArrayIfImpl
{ {
@ -1013,43 +1128,94 @@ private:
const ColumnString * col_then = typeid_cast<const ColumnString *>(col_then_untyped); const ColumnString * col_then = typeid_cast<const ColumnString *>(col_then_untyped);
const ColumnString * col_else = typeid_cast<const ColumnString *>(col_else_untyped); const ColumnString * col_else = typeid_cast<const ColumnString *>(col_else_untyped);
const ColumnFixedString * col_then_fixed = typeid_cast<const ColumnFixedString *>(col_then_untyped);
const ColumnFixedString * col_else_fixed = typeid_cast<const ColumnFixedString *>(col_else_untyped);
const ColumnConstString * col_then_const = typeid_cast<const ColumnConstString *>(col_then_untyped); const ColumnConstString * col_then_const = typeid_cast<const ColumnConstString *>(col_then_untyped);
const ColumnConstString * col_else_const = typeid_cast<const ColumnConstString *>(col_else_untyped); const ColumnConstString * col_else_const = typeid_cast<const ColumnConstString *>(col_else_untyped);
if ((col_then || col_then_const) && (col_else || col_else_const)) if ((col_then || col_then_const || col_then_fixed) && (col_else || col_else_const || col_else_fixed))
{ {
ColumnString * col_res = new ColumnString; if (col_then_fixed && col_else_fixed)
block.getByPosition(result).column = col_res; {
/// Результат - FixedString.
ColumnString::Chars_t & res_vec = col_res->getChars(); if (col_then_fixed->getN() != col_else_fixed->getN())
ColumnString::Offsets_t & res_offsets = col_res->getOffsets(); throw Exception("FixedString columns as 'then' and 'else' arguments of function 'if' has different sizes", ErrorCodes::ILLEGAL_COLUMN);
if (col_then && col_else) size_t N = col_then_fixed->getN();
StringIfImpl::vector_vector(
ColumnFixedString * col_res = new ColumnFixedString(N);
block.getByPosition(result).column = col_res;
ColumnFixedString::Chars_t & res_vec = col_res->getChars();
StringIfImpl::vector_fixed_vector_fixed(
cond_col->getData(), cond_col->getData(),
col_then->getChars(), col_then->getOffsets(), col_then_fixed->getChars(),
col_else->getChars(), col_else->getOffsets(), col_else_fixed->getChars(),
res_vec, res_offsets); N,
else if (col_then && col_else_const) res_vec);
StringIfImpl::vector_constant( }
cond_col->getData(),
col_then->getChars(), col_then->getOffsets(),
col_else_const->getData(),
res_vec, res_offsets);
else if (col_then_const && col_else)
StringIfImpl::constant_vector(
cond_col->getData(),
col_then_const->getData(),
col_else->getChars(), col_else->getOffsets(),
res_vec, res_offsets);
else if (col_then_const && col_else_const)
StringIfImpl::constant_constant(
cond_col->getData(),
col_then_const->getData(),
col_else_const->getData(),
res_vec, res_offsets);
else else
return false; {
/// Результат - String.
ColumnString * col_res = new ColumnString;
block.getByPosition(result).column = col_res;
ColumnString::Chars_t & res_vec = col_res->getChars();
ColumnString::Offsets_t & res_offsets = col_res->getOffsets();
if (col_then && col_else)
StringIfImpl::vector_vector(
cond_col->getData(),
col_then->getChars(), col_then->getOffsets(),
col_else->getChars(), col_else->getOffsets(),
res_vec, res_offsets);
else if (col_then && col_else_const)
StringIfImpl::vector_constant(
cond_col->getData(),
col_then->getChars(), col_then->getOffsets(),
col_else_const->getData(),
res_vec, res_offsets);
else if (col_then_const && col_else)
StringIfImpl::constant_vector(
cond_col->getData(),
col_then_const->getData(),
col_else->getChars(), col_else->getOffsets(),
res_vec, res_offsets);
else if (col_then_const && col_else_const)
StringIfImpl::constant_constant(
cond_col->getData(),
col_then_const->getData(),
col_else_const->getData(),
res_vec, res_offsets);
else if (col_then && col_else_fixed)
StringIfImpl::vector_vector_fixed(
cond_col->getData(),
col_then->getChars(), col_then->getOffsets(),
col_else_fixed->getChars(), col_else_fixed->getN(),
res_vec, res_offsets);
else if (col_then_fixed && col_else)
StringIfImpl::vector_fixed_vector(
cond_col->getData(),
col_then_fixed->getChars(), col_then_fixed->getN(),
col_else->getChars(), col_else->getOffsets(),
res_vec, res_offsets);
else if (col_then_const && col_else_fixed)
StringIfImpl::constant_vector_fixed(
cond_col->getData(),
col_then_const->getData(),
col_else_fixed->getChars(), col_else_fixed->getN(),
res_vec, res_offsets);
else if (col_then_fixed && col_else_const)
StringIfImpl::vector_fixed_constant(
cond_col->getData(),
col_then_fixed->getChars(), col_then_fixed->getN(),
col_else_const->getData(),
res_vec, res_offsets);
else
return false;
}
return true; return true;
} }
@ -1151,9 +1317,29 @@ public:
} }
else if (arguments[1]->getName() != arguments[2]->getName()) else if (arguments[1]->getName() != arguments[2]->getName())
{ {
throw Exception("Incompatible second and third arguments for function " + getName() + ": " const DataTypeString * type_string1 = typeid_cast<const DataTypeString *>(arguments[1].get());
+ arguments[1]->getName() + " and " + arguments[2]->getName(), const DataTypeString * type_string2 = typeid_cast<const DataTypeString *>(arguments[2].get());
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); const DataTypeFixedString * type_fixed_string1 = typeid_cast<const DataTypeFixedString *>(arguments[1].get());
const DataTypeFixedString * type_fixed_string2 = typeid_cast<const DataTypeFixedString *>(arguments[2].get());
if (type_fixed_string1 && type_fixed_string2)
{
if (type_fixed_string1->getN() != type_fixed_string2->getN())
throw Exception("FixedString types as 'then' and 'else' arguments of function 'if' has different sizes",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return new DataTypeFixedString(type_fixed_string1->getN());
}
else if ((type_string1 || type_fixed_string1) && (type_string2 || type_fixed_string2))
{
return new DataTypeString;
}
throw Exception{
"Incompatible second and third arguments for function " + getName() + ": " +
arguments[1]->getName() + " and " + arguments[2]->getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
} }
return arguments[1]; return arguments[1];

View File

@ -1018,7 +1018,7 @@ public:
* Если функция неприменима для данных аргументов - кинуть исключение. * Если функция неприменима для данных аргументов - кинуть исключение.
* Для неконстантных столбцов arguments[i].column = nullptr. * Для неконстантных столбцов arguments[i].column = nullptr.
*/ */
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments, void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type, DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites) std::vector<ExpressionAction> & out_prerequisites)
{ {
@ -1099,7 +1099,7 @@ public:
private: private:
template <typename T> template <typename T>
bool getSizeTyped(const ColumnWithNameAndType & column, size_t & out_size) bool getSizeTyped(const ColumnWithTypeAndName & column, size_t & out_size)
{ {
if (!typeid_cast<const typename DataTypeFromFieldType<T>::Type *>(&*column.type)) if (!typeid_cast<const typename DataTypeFromFieldType<T>::Type *>(&*column.type))
return false; return false;
@ -1113,7 +1113,7 @@ private:
return true; return true;
} }
size_t getSize(const ColumnWithNameAndType & column) size_t getSize(const ColumnWithTypeAndName & column)
{ {
size_t res; size_t res;
if (getSizeTyped<UInt8>(column, res) || if (getSizeTyped<UInt8>(column, res) ||

View File

@ -18,6 +18,7 @@
#include <DB/Dictionaries/FlatDictionary.h> #include <DB/Dictionaries/FlatDictionary.h>
#include <DB/Dictionaries/HashedDictionary.h> #include <DB/Dictionaries/HashedDictionary.h>
#include <DB/Dictionaries/CacheDictionary.h> #include <DB/Dictionaries/CacheDictionary.h>
#include <DB/Dictionaries/RangeHashedDictionary.h>
#include <statdaemons/ext/range.hpp> #include <statdaemons/ext/range.hpp>
@ -755,10 +756,10 @@ public:
private: private:
DataTypePtr getReturnType(const DataTypes & arguments) const override DataTypePtr getReturnType(const DataTypes & arguments) const override
{ {
if (arguments.size() != 3) if (arguments.size() != 3 && arguments.size() != 4)
throw Exception{ throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " "Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 3.", + toString(arguments.size()) + ", should be 3 or 4.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
}; };
@ -789,6 +790,15 @@ private:
}; };
} }
if (arguments.size() == 4 && !typeid_cast<const DataTypeDate *>(arguments[3].get()))
{
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
return new DataTypeString; return new DataTypeString;
} }
@ -806,7 +816,8 @@ private:
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) && if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) && !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr)) !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{ throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(), "Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE ErrorCodes::UNKNOWN_TYPE
@ -814,13 +825,20 @@ private:
} }
template <typename DictionaryType> template <typename DictionaryType>
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, bool executeDispatch(
const IDictionary * const dictionary) Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{ {
const auto dict = typeid_cast<const DictionaryType *>(dictionary); const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict) if (!dict)
return false; return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get()); const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col) if (!attr_name_col)
throw Exception{ throw Exception{
@ -835,13 +853,16 @@ private:
{ {
const auto out = new ColumnString; const auto out = new ColumnString;
block.getByPosition(result).column = out; block.getByPosition(result).column = out;
dictionary->getString(attr_name, id_col->getData(), out); dict->getString(attr_name, id_col->getData(), out);
} }
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))
{ {
const PODArray<UInt64> ids(1, id_col->getData());
auto out = std::make_unique<ColumnString>();
dict->getString(attr_name, ids, out.get());
block.getByPosition(result).column = new ColumnConst<String>{ block.getByPosition(result).column = new ColumnConst<String>{
id_col->size(), id_col->size(), out->getDataAtWithTerminatingZero(0).toString()
dictionary->getString(attr_name, id_col->getData())
}; };
} }
else else
@ -855,6 +876,109 @@ private:
return true; return true;
} }
template <typename DictionaryType>
bool executeDispatchRange(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 4)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
const auto date_col_untyped = block.getByPosition(arguments[3]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
executeRange(block, result, dict, attr_name, id_col, date_col_untyped);
else if (const auto id_col = typeid_cast<const ColumnConst<UInt64> *>(id_col_untyped))
executeRange(block, result, dict, attr_name, id_col, date_col_untyped);
else
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
}
return true;
}
template <typename DictionaryType>
void executeRange(
Block & block, const size_t result, const DictionaryType * const dictionary, const std::string & attr_name,
const ColumnVector<UInt64> * const id_col, const IColumn * const date_col_untyped)
{
if (const auto date_col = typeid_cast<const ColumnVector<UInt16> *>(date_col_untyped))
{
const auto out = new ColumnString;
block.getByPosition(result).column = out;
dictionary->getString(attr_name, id_col->getData(), date_col->getData(), out);
}
else if (const auto date_col = typeid_cast<const ColumnConst<UInt16> *>(date_col_untyped))
{
auto out = new ColumnString;
block.getByPosition(result).column = out;
const PODArray<UInt16> dates(id_col->size(), date_col->getData());
dictionary->getString(attr_name, id_col->getData(), dates, out);
}
else
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN
};
}
}
template <typename DictionaryType>
void executeRange(
Block & block, const size_t result, const DictionaryType * const dictionary, const std::string & attr_name,
const ColumnConst<UInt64> * const id_col, const IColumn * const date_col_untyped)
{
if (const auto date_col = typeid_cast<const ColumnVector<UInt16> *>(date_col_untyped))
{
const auto out = new ColumnString;
block.getByPosition(result).column = out;
const PODArray<UInt64> ids(date_col->size(), id_col->getData());
dictionary->getString(attr_name, ids, date_col->getData(), out);
}
else if (const auto date_col = typeid_cast<const ColumnConst<UInt16> *>(date_col_untyped))
{
const PODArray<UInt64> ids(1, id_col->getData());
const PODArray<UInt16> dates(1, date_col->getData());
auto out = std::make_unique<ColumnString>();
dictionary->getString(attr_name, ids, dates, out.get());
block.getByPosition(result).column = new ColumnConst<String>{
id_col->size(), out->getDataAtWithTerminatingZero(0).toString()
};
}
else
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN
};
}
}
const ExternalDictionaries & dictionaries; const ExternalDictionaries & dictionaries;
}; };
@ -863,14 +987,20 @@ template <typename DataType> struct DictGetTraits;
#define DECLARE_DICT_GET_TRAITS(TYPE, DATA_TYPE) \ #define DECLARE_DICT_GET_TRAITS(TYPE, DATA_TYPE) \
template <> struct DictGetTraits<DATA_TYPE>\ template <> struct DictGetTraits<DATA_TYPE>\
{\ {\
static TYPE get(const IDictionary * const dict, const std::string & name, const IDictionary::id_t id)\ template <typename DictionaryType>\
{\ static void get(\
return dict->get##TYPE(name, id);\ const DictionaryType * const dict, const std::string & name, const PODArray<UInt64> & ids,\
}\ PODArray<TYPE> & out)\
static void get(const IDictionary * const dict, const std::string & name, const PODArray<IDictionary::id_t> & ids, PODArray<TYPE> & out)\
{\ {\
dict->get##TYPE(name, ids, out);\ dict->get##TYPE(name, ids, out);\
}\ }\
template <typename DictionaryType>\
static void get(\
const DictionaryType * const dict, const std::string & name, const PODArray<UInt64> & ids,\
const PODArray<UInt16> & dates, PODArray<TYPE> & out)\
{\
dict->get##TYPE(name, ids, dates, out);\
}\
}; };
DECLARE_DICT_GET_TRAITS(UInt8, DataTypeUInt8) DECLARE_DICT_GET_TRAITS(UInt8, DataTypeUInt8)
DECLARE_DICT_GET_TRAITS(UInt16, DataTypeUInt16) DECLARE_DICT_GET_TRAITS(UInt16, DataTypeUInt16)
@ -906,10 +1036,10 @@ public:
private: private:
DataTypePtr getReturnType(const DataTypes & arguments) const override DataTypePtr getReturnType(const DataTypes & arguments) const override
{ {
if (arguments.size() != 3) if (arguments.size() != 3 && arguments.size() != 4)
throw Exception{ throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " "Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 3.", + toString(arguments.size()) + ", should be 3 or 4.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
}; };
@ -940,6 +1070,15 @@ private:
}; };
} }
if (arguments.size() == 4 && !typeid_cast<const DataTypeDate *>(arguments[3].get()))
{
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
return new DataType; return new DataType;
} }
@ -957,7 +1096,8 @@ private:
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) && if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) && !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr)) !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{ throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(), "Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE ErrorCodes::UNKNOWN_TYPE
@ -966,12 +1106,19 @@ private:
template <typename DictionaryType> template <typename DictionaryType>
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
const IDictionary * const dictionary) const IDictionaryBase * const dictionary)
{ {
const auto dict = typeid_cast<const DictionaryType *>(dictionary); const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict) if (!dict)
return false; return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get()); const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col) if (!attr_name_col)
throw Exception{ throw Exception{
@ -992,14 +1139,15 @@ private:
const auto size = ids.size(); const auto size = ids.size();
data.resize(size); data.resize(size);
DictGetTraits<DataType>::get(dictionary, attr_name, ids, data); DictGetTraits<DataType>::get(dict, attr_name, ids, data);
} }
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))
{ {
block.getByPosition(result).column = new ColumnConst<Type>{ const PODArray<UInt64> ids(1, id_col->getData());
id_col->size(), PODArray<Type> data(1);
DictGetTraits<DataType>::get(dictionary, attr_name, id_col->getData()) DictGetTraits<DataType>::get(dict, attr_name, ids, data);
};
block.getByPosition(result).column = new ColumnConst<Type>{id_col->size(), data.front()};
} }
else else
{ {
@ -1012,6 +1160,120 @@ private:
return true; return true;
} }
template <typename DictionaryType>
bool executeDispatchRange(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 4)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
const auto date_col_untyped = block.getByPosition(arguments[3]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
executeRange(block, result, dict, attr_name, id_col, date_col_untyped);
else if (const auto id_col = typeid_cast<const ColumnConst<UInt64> *>(id_col_untyped))
executeRange(block, result, dict, attr_name, id_col, date_col_untyped);
else
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
}
return true;
}
template <typename DictionaryType>
void executeRange(
Block & block, const size_t result, const DictionaryType * const dictionary, const std::string & attr_name,
const ColumnVector<UInt64> * const id_col, const IColumn * const date_col_untyped)
{
if (const auto date_col = typeid_cast<const ColumnVector<UInt16> *>(date_col_untyped))
{
const auto size = id_col->size();
const auto & ids = id_col->getData();
const auto & dates = date_col->getData();
const auto out = new ColumnVector<Type>{size};
block.getByPosition(result).column = out;
auto & data = out->getData();
DictGetTraits<DataType>::get(dictionary, attr_name, ids, dates, data);
}
else if (const auto date_col = typeid_cast<const ColumnConst<UInt16> *>(date_col_untyped))
{
const auto size = id_col->size();
const auto & ids = id_col->getData();
const PODArray<UInt16> dates(size, date_col->getData());
const auto out = new ColumnVector<Type>{size};
block.getByPosition(result).column = out;
auto & data = out->getData();
DictGetTraits<DataType>::get(dictionary, attr_name, ids, dates, data);
}
else
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN
};
}
}
template <typename DictionaryType>
void executeRange(
Block & block, const size_t result, const DictionaryType * const dictionary, const std::string & attr_name,
const ColumnConst<UInt64> * const id_col, const IColumn * const date_col_untyped)
{
if (const auto date_col = typeid_cast<const ColumnVector<UInt16> *>(date_col_untyped))
{
const auto size = date_col->size();
const PODArray<UInt64> ids(size, id_col->getData());
const auto & dates = date_col->getData();
const auto out = new ColumnVector<Type>{size};
block.getByPosition(result).column = out;
auto & data = out->getData();
DictGetTraits<DataType>::get(dictionary, attr_name, ids, dates, data);
}
else if (const auto date_col = typeid_cast<const ColumnConst<UInt16> *>(date_col_untyped))
{
const PODArray<UInt64> ids(1, id_col->getData());
const PODArray<UInt16> dates(1, date_col->getData());
PODArray<Type> data(1);
DictGetTraits<DataType>::get(dictionary, attr_name, ids, dates, data);
block.getByPosition(result).column = new ColumnConst<Type>{id_col->size(), data.front()};
}
else
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN
};
}
}
const ExternalDictionaries & dictionaries; const ExternalDictionaries & dictionaries;
}; };
@ -1090,12 +1352,6 @@ private:
auto dict = dictionaries.getDictionary(dict_name_col->getData()); auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get(); const auto dict_ptr = dict.get();
if (!dict->hasHierarchy())
throw Exception{
"Dictionary does not have a hierarchy",
ErrorCodes::UNSUPPORTED_METHOD
};
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) && if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) && !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr)) !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
@ -1107,16 +1363,19 @@ private:
template <typename DictionaryType> template <typename DictionaryType>
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
const IDictionary * const dictionary) const IDictionaryBase * const dictionary)
{ {
const auto dict = typeid_cast<const DictionaryType *>(dictionary); const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict) if (!dict)
return false; return false;
const auto id_col_untyped = block.getByPosition(arguments[1]).column.get(); if (!dict->hasHierarchy())
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped)) throw Exception{
{ "Dictionary does not have a hierarchy",
const auto & in = id_col->getData(); ErrorCodes::UNSUPPORTED_METHOD
};
const auto get_hierarchies = [&] (const PODArray<UInt64> & in, PODArray<UInt64> & out, PODArray<UInt64> & offsets) {
const auto size = in.size(); const auto size = in.size();
/// copy of `in` array /// copy of `in` array
@ -1151,18 +1410,12 @@ private:
break; break;
/// translate all non-zero identifiers at once /// translate all non-zero identifiers at once
dictionary->toParent(*in_array, *out_array); dict->toParent(*in_array, *out_array);
/// we're going to use the `in_array` from this iteration as `out_array` on the next one /// we're going to use the `in_array` from this iteration as `out_array` on the next one
std::swap(in_array, out_array); std::swap(in_array, out_array);
} }
const auto backend = new ColumnVector<UInt64>;
const auto array = new ColumnArray{backend};
block.getByPosition(result).column = array;
auto & out = backend->getData();
auto & offsets = array->getOffsets();
out.reserve(total_count); out.reserve(total_count);
offsets.resize(size); offsets.resize(size);
@ -1172,21 +1425,29 @@ private:
out.insert_assume_reserved(std::begin(ids), std::end(ids)); out.insert_assume_reserved(std::begin(ids), std::end(ids));
offsets[i] = out.size(); offsets[i] = out.size();
} }
};
const auto id_col_untyped = block.getByPosition(arguments[1]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
{
const auto & in = id_col->getData();
const auto backend = new ColumnVector<UInt64>;
const auto array = new ColumnArray{backend};
block.getByPosition(result).column = array;
get_hierarchies(in, backend->getData(), array->getOffsets());
} }
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))
{ {
Array res; const PODArray<UInt64> in(1, id_col->getData());
const auto backend = new ColumnVector<UInt64>;
const auto array = new ColumnArray{backend};
IDictionary::id_t cur = id_col->getData(); get_hierarchies(in, backend->getData(), array->getOffsets());
while (cur)
{
res.push_back(cur);
cur = dictionary->toParent(cur);
}
block.getByPosition(result).column = new ColumnConstArray{ block.getByPosition(result).column = new ColumnConstArray{
id_col->size(), id_col->size(),
res, (*array)[0].get<Array>(),
new DataTypeArray{new DataTypeUInt64} new DataTypeArray{new DataTypeUInt64}
}; };
} }
@ -1280,12 +1541,6 @@ private:
auto dict = dictionaries.getDictionary(dict_name_col->getData()); auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get(); const auto dict_ptr = dict.get();
if (!dict->hasHierarchy())
throw Exception{
"Dictionary does not have a hierarchy",
ErrorCodes::UNSUPPORTED_METHOD
};
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) && if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) && !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr)) !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
@ -1297,19 +1552,25 @@ private:
template <typename DictionaryType> template <typename DictionaryType>
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
const IDictionary * const dictionary) const IDictionaryBase * const dictionary)
{ {
const auto dict = typeid_cast<const DictionaryType *>(dictionary); const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict) if (!dict)
return false; return false;
if (!dict->hasHierarchy())
throw Exception{
"Dictionary does not have a hierarchy",
ErrorCodes::UNSUPPORTED_METHOD
};
const auto child_id_col_untyped = block.getByPosition(arguments[1]).column.get(); const auto child_id_col_untyped = block.getByPosition(arguments[1]).column.get();
const auto ancestor_id_col_untyped = block.getByPosition(arguments[2]).column.get(); const auto ancestor_id_col_untyped = block.getByPosition(arguments[2]).column.get();
if (const auto child_id_col = typeid_cast<const ColumnVector<UInt64> *>(child_id_col_untyped)) if (const auto child_id_col = typeid_cast<const ColumnVector<UInt64> *>(child_id_col_untyped))
execute(block, result, dictionary, child_id_col, ancestor_id_col_untyped); execute(block, result, dict, child_id_col, ancestor_id_col_untyped);
else if (const auto child_id_col = typeid_cast<const ColumnConst<UInt64> *>(child_id_col_untyped)) else if (const auto child_id_col = typeid_cast<const ColumnConst<UInt64> *>(child_id_col_untyped))
execute(block, result, dictionary, child_id_col, ancestor_id_col_untyped); execute(block, result, dict, child_id_col, ancestor_id_col_untyped);
else else
throw Exception{ throw Exception{
"Illegal column " + child_id_col_untyped->getName() "Illegal column " + child_id_col_untyped->getName()

View File

@ -3,6 +3,8 @@
#include <openssl/md5.h> #include <openssl/md5.h>
#include <openssl/sha.h> #include <openssl/sha.h>
#include <city.h> #include <city.h>
#include <farmhash.h>
#include <metrohash.h>
#include <Poco/ByteOrder.h> #include <Poco/ByteOrder.h>
@ -397,11 +399,12 @@ UInt64 toInteger<Float64>(Float64 x)
} }
class FunctionCityHash64 : public IFunction template <typename Impl>
class FunctionNeighbourhoodHash64 : public IFunction
{ {
public: public:
static constexpr auto name = "cityHash64"; static constexpr auto name = Impl::name;
static IFunction * create(const Context & context) { return new FunctionCityHash64; }; static IFunction * create(const Context & context) { return new FunctionNeighbourhoodHash64; };
private: private:
template <typename FromType, bool first> template <typename FromType, bool first>
@ -417,12 +420,12 @@ private:
if (first) if (first)
vec_to[i] = h; vec_to[i] = h;
else else
vec_to[i] = Hash128to64(uint128(vec_to[i], h)); vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], h));
} }
} }
else if (const ColumnConst<FromType> * col_from = typeid_cast<const ColumnConst<FromType> *>(column)) else if (const ColumnConst<FromType> * col_from = typeid_cast<const ColumnConst<FromType> *>(column))
{ {
UInt64 hash = IntHash64Impl::apply(toInteger(col_from->getData())); const UInt64 hash = IntHash64Impl::apply(toInteger(col_from->getData()));
size_t size = vec_to.size(); size_t size = vec_to.size();
if (first) if (first)
{ {
@ -431,7 +434,7 @@ private:
else else
{ {
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
vec_to[i] = Hash128to64(uint128(vec_to[i], hash)); vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], hash));
} }
} }
else else
@ -451,13 +454,13 @@ private:
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
UInt64 h = CityHash64( const UInt64 h = Impl::Hash64(
reinterpret_cast<const char *>(&data[i == 0 ? 0 : offsets[i - 1]]), reinterpret_cast<const char *>(&data[i == 0 ? 0 : offsets[i - 1]]),
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1])); i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
if (first) if (first)
vec_to[i] = h; vec_to[i] = h;
else else
vec_to[i] = Hash128to64(uint128(vec_to[i], h)); vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], h));
} }
} }
else if (const ColumnFixedString * col_from = typeid_cast<const ColumnFixedString *>(column)) else if (const ColumnFixedString * col_from = typeid_cast<const ColumnFixedString *>(column))
@ -467,17 +470,17 @@ private:
size_t size = data.size() / n; size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
UInt64 h = CityHash64(reinterpret_cast<const char *>(&data[i * n]), n); const UInt64 h = Impl::Hash64(reinterpret_cast<const char *>(&data[i * n]), n);
if (first) if (first)
vec_to[i] = h; vec_to[i] = h;
else else
vec_to[i] = Hash128to64(uint128(vec_to[i], h)); vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], h));
} }
} }
else if (const ColumnConstString * col_from = typeid_cast<const ColumnConstString *>(column)) else if (const ColumnConstString * col_from = typeid_cast<const ColumnConstString *>(column))
{ {
UInt64 hash = CityHash64(col_from->getData().data(), col_from->getData().size()); const UInt64 hash = Impl::Hash64(col_from->getData().data(), col_from->getData().size());
size_t size = vec_to.size(); const size_t size = vec_to.size();
if (first) if (first)
{ {
vec_to.assign(size, hash); vec_to.assign(size, hash);
@ -486,7 +489,7 @@ private:
{ {
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
vec_to[i] = Hash128to64(uint128(vec_to[i], hash)); vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], hash));
} }
} }
} }
@ -505,26 +508,26 @@ private:
{ {
const IColumn * nested_column = &col_from->getData(); const IColumn * nested_column = &col_from->getData();
const ColumnArray::Offsets_t & offsets = col_from->getOffsets(); const ColumnArray::Offsets_t & offsets = col_from->getOffsets();
size_t nested_size = nested_column->size(); const size_t nested_size = nested_column->size();
ColumnUInt64::Container_t vec_temp(nested_size); ColumnUInt64::Container_t vec_temp(nested_size);
executeAny<true>(nested_type, nested_column, vec_temp); executeAny<true>(nested_type, nested_column, vec_temp);
size_t size = offsets.size(); const size_t size = offsets.size();
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
size_t begin = i == 0 ? 0 : offsets[i - 1]; const size_t begin = i == 0 ? 0 : offsets[i - 1];
size_t end = offsets[i]; const size_t end = offsets[i];
UInt64 h = IntHash64Impl::apply(end - begin); UInt64 h = IntHash64Impl::apply(end - begin);
if (first) if (first)
vec_to[i] = h; vec_to[i] = h;
else else
vec_to[i] = Hash128to64(uint128(vec_to[i], h)); vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], h));
for (size_t j = begin; j < end; ++j) for (size_t j = begin; j < end; ++j)
vec_to[i] = Hash128to64(uint128(vec_to[i], vec_temp[j])); vec_to[i] = Impl::Hash128to64(typename Impl::uint128_t(vec_to[i], vec_temp[j]));
} }
} }
else if (const ColumnConstArray * col_from = typeid_cast<const ColumnConstArray *>(column)) else if (const ColumnConstArray * col_from = typeid_cast<const ColumnConstArray *>(column))
@ -592,7 +595,7 @@ public:
for (size_t i = 0; i < arguments.size(); ++i) for (size_t i = 0; i < arguments.size(); ++i)
{ {
const ColumnWithNameAndType & column = block.getByPosition(arguments[i]); const ColumnWithTypeAndName & column = block.getByPosition(arguments[i]);
const IDataType * from_type = &*column.type; const IDataType * from_type = &*column.type;
const IColumn * icolumn = &*column.column; const IColumn * icolumn = &*column.column;
@ -702,7 +705,7 @@ public:
"Illegal type " + first_arg->getName() + " of argument of function " + getName(), "Illegal type " + first_arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
}; };
if (arg_count == 2) if (arg_count == 2)
{ {
const auto second_arg = arguments.back().get(); const auto second_arg = arguments.back().get();
@ -816,18 +819,57 @@ private:
struct NameHalfMD5 { static constexpr auto name = "halfMD5"; }; struct NameHalfMD5 { static constexpr auto name = "halfMD5"; };
struct NameSipHash64 { static constexpr auto name = "sipHash64"; }; struct NameSipHash64 { static constexpr auto name = "sipHash64"; };
struct NameCityHash64 { static constexpr auto name = "cityHash64"; };
struct NameIntHash32 { static constexpr auto name = "intHash32"; }; struct NameIntHash32 { static constexpr auto name = "intHash32"; };
struct NameIntHash64 { static constexpr auto name = "intHash64"; }; struct NameIntHash64 { static constexpr auto name = "intHash64"; };
typedef FunctionStringHash64<HalfMD5Impl, NameHalfMD5> FunctionHalfMD5; struct ImplCityHash64
typedef FunctionStringHash64<SipHash64Impl, NameSipHash64> FunctionSipHash64; {
typedef FunctionIntHash<IntHash32Impl, NameIntHash32> FunctionIntHash32; static constexpr auto name = "cityHash64";
typedef FunctionIntHash<IntHash64Impl, NameIntHash64> FunctionIntHash64; using uint128_t = uint128;
typedef FunctionStringHashFixedString<MD5Impl> FunctionMD5;
typedef FunctionStringHashFixedString<SHA1Impl> FunctionSHA1; static auto Hash128to64(const uint128_t & x) { return ::Hash128to64(x); }
typedef FunctionStringHashFixedString<SHA224Impl> FunctionSHA224; static auto Hash64(const char * const s, const std::size_t len) { return CityHash64(s, len); }
typedef FunctionStringHashFixedString<SHA256Impl> FunctionSHA256; };
typedef FunctionStringHashFixedString<SipHash128Impl> FunctionSipHash128;
struct ImplFarmHash64
{
static constexpr auto name = "farmHash64";
using uint128_t = farmhash::uint128_t;
static auto Hash128to64(const uint128_t & x) { return farmhash::Hash128to64(x); }
static auto Hash64(const char * const s, const std::size_t len) { return farmhash::Hash64(s, len); }
};
struct ImplMetroHash64
{
static constexpr auto name = "metroHash64";
using uint128_t = uint128;
static auto Hash128to64(const uint128_t & x) { return ::Hash128to64(x); }
static auto Hash64(const char * const s, const std::size_t len)
{
union {
std::uint64_t u64;
std::uint8_t u8[sizeof(u64)];
};
metrohash64_1(reinterpret_cast<const std::uint8_t *>(s), len, 0, u8);
return u64;
}
};
using FunctionHalfMD5 = FunctionStringHash64<HalfMD5Impl, NameHalfMD5>;
using FunctionSipHash64 = FunctionStringHash64<SipHash64Impl, NameSipHash64>;
using FunctionIntHash32 = FunctionIntHash<IntHash32Impl, NameIntHash32>;
using FunctionIntHash64 = FunctionIntHash<IntHash64Impl, NameIntHash64>;
using FunctionMD5 = FunctionStringHashFixedString<MD5Impl>;
using FunctionSHA1 = FunctionStringHashFixedString<SHA1Impl>;
using FunctionSHA224 = FunctionStringHashFixedString<SHA224Impl>;
using FunctionSHA256 = FunctionStringHashFixedString<SHA256Impl>;
using FunctionSipHash128 = FunctionStringHashFixedString<SipHash128Impl>;
using FunctionCityHash64 = FunctionNeighbourhoodHash64<ImplCityHash64>;
using FunctionFarmHash64 = FunctionNeighbourhoodHash64<ImplFarmHash64>;
using FunctionMetroHash64 = FunctionNeighbourhoodHash64<ImplMetroHash64>;
} }

View File

@ -586,7 +586,7 @@ public:
arguments[0] = new DataTypeExpression(nested_types); arguments[0] = new DataTypeExpression(nested_types);
} }
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments, void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type, DataTypePtr & out_return_type,
ExpressionActions::Actions & out_prerequisites) ExpressionActions::Actions & out_prerequisites)
{ {
@ -725,7 +725,7 @@ public:
column_first_array = column_array; column_first_array = column_array;
} }
temp_block.insert(ColumnWithNameAndType( temp_block.insert(ColumnWithTypeAndName(
column_array->getDataPtr(), column_array->getDataPtr(),
argument_type, argument_type,
argument_name)); argument_name));
@ -745,7 +745,7 @@ public:
if (argument_names.count(name)) if (argument_names.count(name))
continue; continue;
ColumnWithNameAndType replicated_column = block.getByPosition(prerequisites[prerequisite_index]); ColumnWithTypeAndName replicated_column = block.getByPosition(prerequisites[prerequisite_index]);
replicated_column.name = name; replicated_column.name = name;
replicated_column.column = typeid_cast<ColumnArray &>(*replicated_column.column).getDataPtr(); replicated_column.column = typeid_cast<ColumnArray &>(*replicated_column.column).getDataPtr();

View File

@ -453,7 +453,7 @@ public:
return name; return name;
} }
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments, void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type, DataTypePtr & out_return_type,
ExpressionActions::Actions & out_prerequisites) ExpressionActions::Actions & out_prerequisites)
{ {
@ -534,6 +534,36 @@ public:
}; };
class FunctionIdentity : public IFunction
{
public:
static constexpr auto name = "identity";
static IFunction * create(const Context & context) { return new FunctionIdentity; }
/// Получить имя функции.
String getName() const
{
return name;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const
{
if (arguments.size() != 1)
throw Exception("Function " + getName() + " requires exactly one argument.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return arguments.front()->clone();
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
block.getByPosition(result).column = block.getByPosition(arguments.front()).column;
}
};
class FunctionArrayJoin : public IFunction class FunctionArrayJoin : public IFunction
{ {
public: public:

View File

@ -132,7 +132,8 @@ public:
const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column); const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column);
if (!col) if (!col)
throw Exception("Illegal column " + col->getName() + " of first argument of function " + getName() + ". Must be constant string.", throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);
const String & sep_str = col->getData(); const String & sep_str = col->getData();
@ -198,7 +199,8 @@ public:
const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column); const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column);
if (!col) if (!col)
throw Exception("Illegal column " + col->getName() + " of first argument of function " + getName() + ". Must be constant string.", throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);
sep = col->getData(); sep = col->getData();
@ -264,7 +266,8 @@ public:
const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[1]).column); const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[1]).column);
if (!col) if (!col)
throw Exception("Illegal column " + col->getName() + " of first argument of function " + getName() + ". Must be constant string.", throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);
re = Regexps::get<false, false>(col->getData()); re = Regexps::get<false, false>(col->getData());

View File

@ -5,7 +5,7 @@
#include <DB/Core/Names.h> #include <DB/Core/Names.h>
#include <DB/Core/Block.h> #include <DB/Core/Block.h>
#include <DB/Core/ColumnNumbers.h> #include <DB/Core/ColumnNumbers.h>
#include <DB/Core/ColumnsWithNameAndType.h> #include <DB/Core/ColumnsWithTypeAndName.h>
#include <DB/DataTypes/IDataType.h> #include <DB/DataTypes/IDataType.h>
@ -54,7 +54,7 @@ public:
* Осмысленные типы элементов в out_prerequisites: APPLY_FUNCTION, ADD_COLUMN. * Осмысленные типы элементов в out_prerequisites: APPLY_FUNCTION, ADD_COLUMN.
*/ */
virtual void getReturnTypeAndPrerequisites( virtual void getReturnTypeAndPrerequisites(
const ColumnsWithNameAndType & arguments, const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type, DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites) std::vector<ExpressionAction> & out_prerequisites)
{ {

View File

@ -85,8 +85,7 @@ inline void writeBoolText(bool x, WriteBuffer & buf)
} }
template <typename T> inline void writeFloatText(double x, WriteBuffer & buf)
void writeFloatText(T x, WriteBuffer & buf)
{ {
char tmp[25]; char tmp[25];
double_conversion::StringBuilder builder{tmp, sizeof(tmp)}; double_conversion::StringBuilder builder{tmp, sizeof(tmp)};
@ -94,7 +93,20 @@ void writeFloatText(T x, WriteBuffer & buf)
const auto result = getDoubleToStringConverter<false>().ToShortest(x, &builder); const auto result = getDoubleToStringConverter<false>().ToShortest(x, &builder);
if (!result) if (!result)
throw Exception("Cannot print float or double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER); throw Exception("Cannot print double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
buf.write(tmp, builder.position());
}
inline void writeFloatText(float x, WriteBuffer & buf)
{
char tmp[25];
double_conversion::StringBuilder builder{tmp, sizeof(tmp)};
const auto result = getDoubleToStringConverter<false>().ToShortestSingle(x, &builder);
if (!result)
throw Exception("Cannot print float number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
buf.write(tmp, builder.position()); buf.write(tmp, builder.position());
} }

View File

@ -374,8 +374,20 @@ struct AggregationMethodConcat
/// См. функцию extractKeysAndPlaceInPoolContiguous. /// См. функцию extractKeysAndPlaceInPoolContiguous.
const StringRef * key_refs = reinterpret_cast<const StringRef *>(value.first.data + value.first.size); const StringRef * key_refs = reinterpret_cast<const StringRef *>(value.first.data + value.first.size);
for (size_t i = 0; i < keys_size; ++i) if (unlikely(0 == value.first.size))
key_columns[i]->insertDataWithTerminatingZero(key_refs[i].data, key_refs[i].size); {
/** Исправление, если все ключи - пустые массивы. Для них в хэш-таблицу записывается StringRef нулевой длины, но с ненулевым указателем.
* Но при вставке в хэш-таблицу, такой StringRef оказывается равен другому ключу нулевой длины,
* у которого указатель на данные может быть любым мусором и использовать его нельзя.
*/
for (size_t i = 0; i < keys_size; ++i)
key_columns[i]->insertDefault();
}
else
{
for (size_t i = 0; i < keys_size; ++i)
key_columns[i]->insertDataWithTerminatingZero(key_refs[i].data, key_refs[i].size);
}
} }
}; };

View File

@ -41,6 +41,7 @@ class Macros;
class Progress; class Progress;
class Clusters; class Clusters;
class QueryLog; class QueryLog;
struct MergeTreeSettings;
/// имя таблицы -> таблица /// имя таблицы -> таблица
@ -92,7 +93,6 @@ private:
std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает. std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
String current_database; /// Текущая БД. String current_database; /// Текущая БД.
String current_query_id; /// Id текущего запроса. String current_query_id; /// Id текущего запроса.
NamesAndTypesList columns; /// Столбцы текущей обрабатываемой таблицы.
Settings settings; /// Настройки выполнения запроса. Settings settings; /// Настройки выполнения запроса.
using ProgressCallback = std::function<void(const Progress & progress)>; using ProgressCallback = std::function<void(const Progress & progress)>;
ProgressCallback progress_callback; /// Колбек для отслеживания прогресса выполнения запроса. ProgressCallback progress_callback; /// Колбек для отслеживания прогресса выполнения запроса.
@ -207,11 +207,6 @@ public:
const Databases & getDatabases() const; const Databases & getDatabases() const;
Databases & getDatabases(); Databases & getDatabases();
/// При работе со списком столбцов, используйте локальный контекст, чтобы никто больше его не менял.
const NamesAndTypesList & getColumns() const { return columns; }
NamesAndTypesList & getColumns() { return columns; }
void setColumns(const NamesAndTypesList & columns_) { columns = columns_; }
Context & getSessionContext(); Context & getSessionContext();
Context & getGlobalContext(); Context & getGlobalContext();
@ -266,8 +261,8 @@ public:
Poco::SharedPtr<Clusters> getClusters() const; Poco::SharedPtr<Clusters> getClusters() const;
Compiler & getCompiler(); Compiler & getCompiler();
QueryLog & getQueryLog(); QueryLog & getQueryLog();
const MergeTreeSettings & getMergeTreeSettings();
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле. /// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const; CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;

View File

@ -4,7 +4,7 @@
#include <DB/DataStreams/IBlockInputStream.h> #include <DB/DataStreams/IBlockInputStream.h>
#include <DB/Interpreters/Settings.h> #include <DB/Interpreters/Settings.h>
#include <DB/Core/Names.h> #include <DB/Core/Names.h>
#include <DB/Core/ColumnWithNameAndType.h> #include <DB/Core/ColumnWithTypeAndName.h>
#include <DB/Core/Block.h> #include <DB/Core/Block.h>
#include <unordered_set> #include <unordered_set>
@ -77,7 +77,7 @@ public:
/// Если result_name_ == "", в качестве имени используется "имя_функции(аргументы через запятую)". /// Если result_name_ == "", в качестве имени используется "имя_функции(аргументы через запятую)".
static ExpressionAction applyFunction(FunctionPtr function_, const std::vector<std::string> & argument_names_, std::string result_name_ = ""); static ExpressionAction applyFunction(FunctionPtr function_, const std::vector<std::string> & argument_names_, std::string result_name_ = "");
static ExpressionAction addColumn(ColumnWithNameAndType added_column_) static ExpressionAction addColumn(ColumnWithTypeAndName added_column_)
{ {
ExpressionAction a; ExpressionAction a;
a.type = ADD_COLUMN; a.type = ADD_COLUMN;
@ -168,11 +168,11 @@ public:
: input_columns(input_columns_), settings(settings_) : input_columns(input_columns_), settings(settings_)
{ {
for (const auto & input_elem : input_columns) for (const auto & input_elem : input_columns)
sample_block.insert(ColumnWithNameAndType(nullptr, input_elem.type, input_elem.name)); sample_block.insert(ColumnWithTypeAndName(nullptr, input_elem.type, input_elem.name));
} }
/// Для константных столбцов в input_columns_ могут содержаться сами столбцы. /// Для константных столбцов в input_columns_ могут содержаться сами столбцы.
ExpressionActions(const ColumnsWithNameAndType & input_columns_, const Settings & settings_) ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Settings & settings_)
: settings(settings_) : settings(settings_)
{ {
for (const auto & input_elem : input_columns_) for (const auto & input_elem : input_columns_)
@ -185,7 +185,7 @@ public:
/// Добавить входной столбец. /// Добавить входной столбец.
/// Название столбца не должно совпадать с названиями промежуточных столбцов, возникающих при вычислении выражения. /// Название столбца не должно совпадать с названиями промежуточных столбцов, возникающих при вычислении выражения.
/// В выражении не должно быть действий PROJECT. /// В выражении не должно быть действий PROJECT.
void addInput(const ColumnWithNameAndType & column); void addInput(const ColumnWithTypeAndName & column);
void addInput(const NameAndTypePair & column); void addInput(const NameAndTypePair & column);
void add(const ExpressionAction & action); void add(const ExpressionAction & action);

View File

@ -46,17 +46,18 @@ typedef std::unordered_map<String, SubqueryForSet> SubqueriesForSets;
class ExpressionAnalyzer : private boost::noncopyable class ExpressionAnalyzer : private boost::noncopyable
{ {
public: public:
ExpressionAnalyzer(const ASTPtr & ast_, const Context & context_, StoragePtr storage_, size_t subquery_depth_ = 0, bool do_global_ = false) ExpressionAnalyzer(
: ast(ast_), context(context_), settings(context.getSettings()), const ASTPtr & ast_,
subquery_depth(subquery_depth_), columns(context.getColumns()), storage(storage_ ? storage_ : getTable()), do_global(do_global_) const Context & context_,
{ StoragePtr storage_,
init(); const NamesAndTypesList & columns_,
} size_t subquery_depth_ = 0,
bool do_global_ = false)
/// columns - список известных столбцов (которых можно достать из таблицы). :
ExpressionAnalyzer(const ASTPtr & ast_, const Context & context_, const NamesAndTypesList & columns_, size_t subquery_depth_ = 0, bool do_global_ = false) ast(ast_), context(context_), settings(context.getSettings()),
: ast(ast_), context(context_), settings(context.getSettings()), subquery_depth(subquery_depth_), columns(columns_),
subquery_depth(subquery_depth_), columns(columns_), storage(getTable()), do_global(do_global_) storage(storage_ ? storage_ : getTable()),
do_global(do_global_)
{ {
init(); init();
} }

View File

@ -42,7 +42,7 @@ private:
mutable std::mutex dictionaries_mutex; mutable std::mutex dictionaries_mutex;
using dictionary_ptr_t = std::shared_ptr<MultiVersion<IDictionary>>; using dictionary_ptr_t = std::shared_ptr<MultiVersion<IDictionaryBase>>;
struct dictionary_info final struct dictionary_info final
{ {
dictionary_ptr_t dict; dictionary_ptr_t dict;
@ -52,7 +52,7 @@ private:
struct failed_dictionary_info final struct failed_dictionary_info final
{ {
std::unique_ptr<IDictionary> dict; std::unique_ptr<IDictionaryBase> dict;
std::chrono::system_clock::time_point next_attempt_time; std::chrono::system_clock::time_point next_attempt_time;
std::uint64_t error_count; std::uint64_t error_count;
}; };
@ -107,7 +107,7 @@ public:
reloading_thread.join(); reloading_thread.join();
} }
MultiVersion<IDictionary>::Version getDictionary(const std::string & name) const; MultiVersion<IDictionaryBase>::Version getDictionary(const std::string & name) const;
}; };
} }

View File

@ -44,7 +44,7 @@ private:
{ {
Block block; Block block;
ColumnWithNameAndType col; ColumnWithTypeAndName col;
col.name = "name"; col.name = "name";
col.type = new DataTypeString; col.type = new DataTypeString;
col.column = col.type->createColumn(); col.column = col.type->createColumn();
@ -78,10 +78,10 @@ private:
column_defaults = table->column_defaults; column_defaults = table->column_defaults;
} }
ColumnWithNameAndType name_column{new ColumnString, new DataTypeString, "name"}; ColumnWithTypeAndName name_column{new ColumnString, new DataTypeString, "name"};
ColumnWithNameAndType type_column{new ColumnString, new DataTypeString, "type" }; ColumnWithTypeAndName type_column{new ColumnString, new DataTypeString, "type" };
ColumnWithNameAndType default_type_column{new ColumnString, new DataTypeString, "default_type" }; ColumnWithTypeAndName default_type_column{new ColumnString, new DataTypeString, "default_type" };
ColumnWithNameAndType default_expression_column{new ColumnString, new DataTypeString, "default_expression" };; ColumnWithTypeAndName default_expression_column{new ColumnString, new DataTypeString, "default_expression" };;
for (const auto column : columns) for (const auto column : columns)
{ {

View File

@ -39,7 +39,7 @@ private:
Block getSampleBlock() Block getSampleBlock()
{ {
ColumnWithNameAndType col; ColumnWithTypeAndName col;
col.name = "result"; col.name = "result";
col.type = new DataTypeUInt8; col.type = new DataTypeUInt8;
col.column = col.type->createColumn(); col.column = col.type->createColumn();
@ -56,7 +56,7 @@ private:
bool res = context.isTableExist(ast.database, ast.table); bool res = context.isTableExist(ast.database, ast.table);
ColumnWithNameAndType col; ColumnWithTypeAndName col;
col.name = "result"; col.name = "result";
col.type = new DataTypeUInt8; col.type = new DataTypeUInt8;
col.column = new ColumnConstUInt8(1, res); col.column = new ColumnConstUInt8(1, res);

View File

@ -33,7 +33,7 @@ public:
* - удалить из запроса все столбцы кроме указанных - используется для удаления ненужных столбцов из подзапросов. * - удалить из запроса все столбцы кроме указанных - используется для удаления ненужных столбцов из подзапросов.
* *
* table_column_names * table_column_names
* - поместить в контекст в качестве известных столбцов только указанные столбцы, а не все столбцы таблицы. * - список доступных столбцов таблицы.
* Используется, например, совместно с указанием input. * Используется, например, совместно с указанием input.
*/ */
@ -42,8 +42,7 @@ public:
const Context & context_, const Context & context_,
QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete,
size_t subquery_depth_ = 0, size_t subquery_depth_ = 0,
BlockInputStreamPtr input = nullptr, BlockInputStreamPtr input = nullptr);
bool is_union_all_head_ = true);
InterpreterSelectQuery( InterpreterSelectQuery(
ASTPtr query_ptr_, ASTPtr query_ptr_,
@ -57,7 +56,7 @@ public:
ASTPtr query_ptr_, ASTPtr query_ptr_,
const Context & context_, const Context & context_,
const Names & required_column_names, const Names & required_column_names,
const NamesAndTypesList & table_column_names, const NamesAndTypesList & table_column_names_,
QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete,
size_t subquery_depth_ = 0, size_t subquery_depth_ = 0,
BlockInputStreamPtr input = nullptr); BlockInputStreamPtr input = nullptr);
@ -76,9 +75,23 @@ public:
DataTypes getReturnTypes(); DataTypes getReturnTypes();
Block getSampleBlock(); Block getSampleBlock();
static Block getSampleBlock(
ASTPtr query_ptr_,
const Context & context_);
private: private:
void init(BlockInputStreamPtr input, const Names & required_column_names = Names(), const NamesAndTypesList & table_column_names = NamesAndTypesList()); /**
void basicInit(BlockInputStreamPtr input, const NamesAndTypesList & table_column_names); * - Оптимизация, если объект создаётся только, чтобы вызвать getSampleBlock(): учитываем только первый SELECT цепочки UNION ALL, потому что
* первый SELECT достаточен для определения нужных столбцов.
*/
struct OnlyAnalyzeTag {};
InterpreterSelectQuery(
OnlyAnalyzeTag,
ASTPtr query_ptr_,
const Context & context_);
void init(BlockInputStreamPtr input, const Names & required_column_names = Names{});
void basicInit(BlockInputStreamPtr input);
void initQueryAnalyzer(); void initQueryAnalyzer();
/// Выполнить один запрос SELECT из цепочки UNION ALL. /// Выполнить один запрос SELECT из цепочки UNION ALL.
@ -96,7 +109,7 @@ private:
// Переименовать столбцы каждого запроса цепочки UNION ALL в такие же имена, как в первом запросе. // Переименовать столбцы каждого запроса цепочки UNION ALL в такие же имена, как в первом запросе.
void renameColumns(); void renameColumns();
/** Из какой таблицы читать. При JOIN, возвращается "левая" таблицы. /** Из какой таблицы читать. При JOIN, возвращается "левая" таблица.
*/ */
void getDatabaseAndTableNames(String & database_name, String & table_name); void getDatabaseAndTableNames(String & database_name, String & table_name);
@ -107,22 +120,43 @@ private:
/// Разные стадии выполнения запроса. /// Разные стадии выполнения запроса.
/// Вынимает данные из таблицы. Возвращает стадию, до которой запрос был обработан в Storage. /// Вынимает данные из таблицы. Возвращает стадию, до которой запрос был обработан в Storage.
QueryProcessingStage::Enum executeFetchColumns(BlockInputStreams & streams); QueryProcessingStage::Enum executeFetchColumns();
void executeWhere(ExpressionActionsPtr expression);
void executeAggregation(ExpressionActionsPtr expression, bool overflow_row, bool final);
void executeMergeAggregated(bool overflow_row, bool final);
void executeTotalsAndHaving(bool has_having, ExpressionActionsPtr expression, bool overflow_row);
void executeHaving(ExpressionActionsPtr expression);
void executeExpression(ExpressionActionsPtr expression);
void executeOrder();
void executeMergeSorted();
void executePreLimit();
void executeUnion();
void executeLimit();
void executeProjection(ExpressionActionsPtr expression);
void executeDistinct(bool before_order, Names columns);
void executeSubqueriesInSetsAndJoins(std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
template <typename Transform>
void transformStreams(Transform && transform)
{
for (auto & stream : streams)
transform(stream);
if (stream_with_non_joined_data)
transform(stream_with_non_joined_data);
}
bool hasNoData() const
{
return streams.empty() && !stream_with_non_joined_data;
}
bool hasMoreThanOneStream() const
{
return streams.size() + (stream_with_non_joined_data ? 1 : 0) > 1;
}
void executeWhere( BlockInputStreams & streams, ExpressionActionsPtr expression);
void executeAggregation( BlockInputStreams & streams, ExpressionActionsPtr expression, bool overflow_row, bool final);
void executeMergeAggregated( BlockInputStreams & streams, bool overflow_row, bool final);
void executeTotalsAndHaving( BlockInputStreams & streams, bool has_having, ExpressionActionsPtr expression, bool overflow_row);
void executeHaving( BlockInputStreams & streams, ExpressionActionsPtr expression);
void executeExpression( BlockInputStreams & streams, ExpressionActionsPtr expression);
void executeOrder( BlockInputStreams & streams);
void executeMergeSorted( BlockInputStreams & streams);
void executePreLimit( BlockInputStreams & streams);
void executeUnion( BlockInputStreams & streams);
void executeLimit( BlockInputStreams & streams);
void executeProjection( BlockInputStreams & streams, ExpressionActionsPtr expression);
void executeDistinct( BlockInputStreams & streams, bool before_order, Names columns);
void executeSubqueriesInSetsAndJoins(BlockInputStreams & streams, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
void ignoreWithTotals(); void ignoreWithTotals();
@ -143,12 +177,28 @@ private:
QueryProcessingStage::Enum to_stage; QueryProcessingStage::Enum to_stage;
size_t subquery_depth; size_t subquery_depth;
std::unique_ptr<ExpressionAnalyzer> query_analyzer; std::unique_ptr<ExpressionAnalyzer> query_analyzer;
NamesAndTypesList table_column_names;
/** Потоки данных.
* Исходные потоки данных получаются в функции executeFetchColumns.
* Затем они преобразуются (оборачиваются в другие потоки) с помощью функций execute*,
* чтобы получить целый конвейер выполнения запроса.
*/
BlockInputStreams streams; BlockInputStreams streams;
/** При выполнении FULL или RIGHT JOIN, здесь будет поток данных, из которого можно прочитать "неприсоединённые" строки.
* Он имеет особое значение, так как чтение из него должно осуществляться после чтения из основных потоков.
* Он подклеивается к основным потокам в UnionBlockInputStream или ParallelAggregatingBlockInputStream.
*/
BlockInputStreamPtr stream_with_non_joined_data;
/// Являемся ли мы первым запросом SELECT цепочки UNION ALL? /// Являемся ли мы первым запросом SELECT цепочки UNION ALL?
bool is_first_select_inside_union_all; bool is_first_select_inside_union_all;
/// Следующий запрос SELECT в цепочке UNION ALL. /// Объект создан только для анализа запроса.
bool only_analyze = false;
/// Следующий запрос SELECT в цепочке UNION ALL, если есть.
std::unique_ptr<InterpreterSelectQuery> next_select_in_union_all; std::unique_ptr<InterpreterSelectQuery> next_select_in_union_all;
/// Таблица, откуда читать данные, если не подзапрос. /// Таблица, откуда читать данные, если не подзапрос.

View File

@ -40,7 +40,7 @@ private:
Block getSampleBlock() Block getSampleBlock()
{ {
ColumnWithNameAndType col; ColumnWithTypeAndName col;
col.name = "statement"; col.name = "statement";
col.type = new DataTypeString; col.type = new DataTypeString;
col.column = col.type->createColumn(); col.column = col.type->createColumn();
@ -59,7 +59,7 @@ private:
formatAST(*context.getCreateQuery(ast.database, ast.table), stream, 0, false, true); formatAST(*context.getCreateQuery(ast.database, ast.table), stream, 0, false, true);
String res = stream.str(); String res = stream.str();
ColumnWithNameAndType col; ColumnWithTypeAndName col;
col.name = "statement"; col.name = "statement";
col.type = new DataTypeString; col.type = new DataTypeString;
col.column = new ColumnConstString(1, res); col.column = new ColumnConstString(1, res);

View File

@ -39,11 +39,16 @@ inline void evaluateMissingDefaults(Block & block,
* we are going to operate on a copy instead of the original block */ * we are going to operate on a copy instead of the original block */
Block copy_block{block}; Block copy_block{block};
/// evaluate default values for defaulted columns /// evaluate default values for defaulted columns
ExpressionAnalyzer{default_expr_list, context, required_columns}.getActions(true)->execute(copy_block); ExpressionAnalyzer{default_expr_list, context, {}, required_columns}.getActions(true)->execute(copy_block);
/// move evaluated columns to the original block /// move evaluated columns to the original block, materializing them at the same time
for (auto & column_name_type : copy_block.getColumns()) for (auto & column_name_type : copy_block.getColumns())
{
if (column_name_type.column->isConst())
column_name_type.column = static_cast<const IColumnConst &>(*column_name_type.column).convertToFullColumn();
block.insert(std::move(column_name_type)); block.insert(std::move(column_name_type));
}
} }
} }

View File

@ -22,7 +22,7 @@ namespace DB
/** pass a dummy column name because ExpressionAnalyzer /** pass a dummy column name because ExpressionAnalyzer
* does not work with no columns so far. */ * does not work with no columns so far. */
ExpressionAnalyzer{ ExpressionAnalyzer{
expr, context, expr, context, {},
{ { "", new DataTypeString } } { { "", new DataTypeString } }
}.getActions(false)->execute(block); }.getActions(false)->execute(block);

View File

@ -32,11 +32,19 @@ public:
/// Переписывает select_expression_list, чтобы вернуть только необходимые столбцы в правильном порядке. /// Переписывает select_expression_list, чтобы вернуть только необходимые столбцы в правильном порядке.
void rewriteSelectExpressionList(const Names & column_names); void rewriteSelectExpressionList(const Names & column_names);
bool isUnionAllHead() const { return prev_union_all.isNull() && !next_union_all.isNull(); }
ASTPtr clone() const override; ASTPtr clone() const override;
/// Получить глубокую копию дерева первого запроса SELECT.
ASTPtr cloneFirstSelect() const;
/// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL. /// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
const IAST * getFormat() const override; const IAST * getFormat() const override;
private:
ASTPtr cloneImpl(bool traverse_union_all) const;
public: public:
bool distinct = false; bool distinct = false;
ASTPtr select_expression_list; ASTPtr select_expression_list;
@ -55,7 +63,10 @@ public:
ASTPtr limit_offset; ASTPtr limit_offset;
ASTPtr limit_length; ASTPtr limit_length;
ASTPtr settings; ASTPtr settings;
ASTPtr next_union_all; /// Следующий запрос SELECT в цепочке UNION ALL, если такой есть /// Предыдущий запрос SELECT в цепочке UNION ALL (не вставляется в children и не клонируется)
ASTPtr prev_union_all;
/// Следующий запрос SELECT в цепочке UNION ALL, если такой есть
ASTPtr next_union_all;
}; };
} }

View File

@ -305,7 +305,7 @@ protected:
size_t rows = 0; size_t rows = 0;
for (size_t i = 0; i < res.columns(); ++i) for (size_t i = 0; i < res.columns(); ++i)
{ {
ColumnWithNameAndType & column = res.getByPosition(i); ColumnWithTypeAndName & column = res.getByPosition(i);
if (column.name == prewhere_column && res.columns() > 1) if (column.name == prewhere_column && res.columns() > 1)
continue; continue;
column.column = column.column->filter(column_name_set.count(column.name) ? post_filter : pre_filter); column.column = column.column->filter(column_name_set.count(column.name) ? post_filter : pre_filter);

View File

@ -5,6 +5,7 @@
#include <DB/Interpreters/ExpressionActions.h> #include <DB/Interpreters/ExpressionActions.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Storages/MergeTree/ActiveDataPartSet.h> #include <DB/Storages/MergeTree/ActiveDataPartSet.h>
#include <DB/Storages/MergeTree/MergeTreeSettings.h>
#include <DB/IO/ReadBufferFromString.h> #include <DB/IO/ReadBufferFromString.h>
#include <DB/IO/WriteBufferFromFile.h> #include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/ReadBufferFromFile.h> #include <DB/IO/ReadBufferFromFile.h>
@ -69,69 +70,6 @@ namespace DB
* - MergeTreeDataMerger * - MergeTreeDataMerger
*/ */
struct MergeTreeSettings
{
/** Настройки слияний. */
/// Опеределяет, насколько разбалансированные объединения мы готовы делать.
/// Чем больше, тем более разбалансированные. Желательно, чтобы было больше, чем 1 / max_parts_to_merge_at_once.
double size_ratio_coefficient_to_merge_parts = 0.25;
/// Сколько за раз сливать кусков.
/// Трудоемкость выбора кусков O(N * max_parts_to_merge_at_once).
size_t max_parts_to_merge_at_once = 10;
/// Но пока суммарный размер кусков слишком маленький (меньше такого количества байт), можно сливать и больше кусков за раз.
/// Это сделано, чтобы быстрее сливать очень уж маленькие куски, которых может быстро накопиться много.
size_t merge_more_parts_if_sum_bytes_is_less_than = 100 * 1024 * 1024;
size_t max_parts_to_merge_at_once_if_small = 100;
/// Куски настолько большого размера объединять нельзя вообще.
size_t max_bytes_to_merge_parts = 10ul * 1024 * 1024 * 1024;
/// Не больше половины потоков одновременно могут выполнять слияния, в которых участвует хоть один кусок хотя бы такого размера.
size_t max_bytes_to_merge_parts_small = 250 * 1024 * 1024;
/// Куски настолько большого размера в сумме, объединять нельзя вообще.
size_t max_sum_bytes_to_merge_parts = 25ul * 1024 * 1024 * 1024;
/// Во столько раз ночью увеличиваем коэффициент.
size_t merge_parts_at_night_inc = 10;
/// Сколько заданий на слияние кусков разрешено одновременно иметь в очереди ReplicatedMergeTree.
size_t max_replicated_merges_in_queue = 6;
/// Через сколько секунд удалять ненужные куски.
time_t old_parts_lifetime = 8 * 60;
/** Настройки вставок. */
/// Если в таблице хотя бы столько активных кусков, искусственно замедлять вставки в таблицу.
size_t parts_to_delay_insert = 150;
/// Если в таблице parts_to_delay_insert + k кусков, спать insert_delay_step^k миллисекунд перед вставкой каждого блока.
/// Таким образом, скорость вставок автоматически замедлится примерно до скорости слияний.
double insert_delay_step = 1.1;
/** Настройки репликации. */
/// Для скольки последних блоков хранить хеши в ZooKeeper.
size_t replicated_deduplication_window = 100;
/// Хранить примерно столько последних записей в логе в ZooKeeper, даже если они никому уже не нужны.
/// Не влияет на работу таблиц; используется только чтобы успеть посмотреть на лог в ZooKeeper глазами прежде, чем его очистят.
size_t replicated_logs_to_keep = 100;
/// Максимальное количество ошибок при загрузке кусков, при котором ReplicatedMergeTree соглашается запускаться.
size_t replicated_max_unexpected_parts = 3;
size_t replicated_max_unexpectedly_merged_parts = 2;
size_t replicated_max_missing_obsolete_parts = 5;
size_t replicated_max_missing_active_parts = 20;
/// Если отношение количества ошибок к общему количеству кусков меньше указанного значения, то всё-равно можно запускаться.
double replicated_max_ratio_of_wrong_parts = 0.05;
};
class MergeTreeData : public ITableDeclaration class MergeTreeData : public ITableDeclaration
{ {
public: public:

View File

@ -69,10 +69,11 @@ private:
ExpressionActionsPtr prewhere_actions, ExpressionActionsPtr prewhere_actions,
const String & prewhere_column, const String & prewhere_column,
const Names & virt_columns, const Names & virt_columns,
const Settings & settings); const Settings & settings,
const Context & context);
/// Создать выражение "Sign == 1". /// Создать выражение "Sign == 1".
void createPositiveSignCondition(ExpressionActionsPtr & out_expression, String & out_column); void createPositiveSignCondition(ExpressionActionsPtr & out_expression, String & out_column, const Context & context);
MarkRanges markRangesFromPkRange(const MergeTreeData::DataPart::Index & index, PKCondition & key_condition, const Settings & settings); MarkRanges markRangesFromPkRange(const MergeTreeData::DataPart::Index & index, PKCondition & key_condition, const Settings & settings);
}; };

View File

@ -92,7 +92,7 @@ public:
/// Все столбцы уже есть в блоке. Будем добавлять значения в конец. /// Все столбцы уже есть в блоке. Будем добавлять значения в конец.
bool append = res.has(it.name); bool append = res.has(it.name);
ColumnWithNameAndType column; ColumnWithTypeAndName column;
column.name = it.name; column.name = it.name;
column.type = it.type; column.type = it.type;
if (append) if (append)
@ -487,7 +487,7 @@ private:
OffsetColumns offset_columns; OffsetColumns offset_columns;
for (size_t i = 0; i < res.columns(); ++i) for (size_t i = 0; i < res.columns(); ++i)
{ {
const ColumnWithNameAndType & column = res.getByPosition(i); const ColumnWithTypeAndName & column = res.getByPosition(i);
if (const ColumnArray * array = typeid_cast<const ColumnArray *>(&*column.column)) if (const ColumnArray * array = typeid_cast<const ColumnArray *>(&*column.column))
{ {
String offsets_name = DataTypeNested::extractNestedTableName(column.name); String offsets_name = DataTypeNested::extractNestedTableName(column.name);
@ -514,7 +514,7 @@ private:
continue; continue;
} }
ColumnWithNameAndType column_to_add; ColumnWithTypeAndName column_to_add;
column_to_add.name = requested_column.name; column_to_add.name = requested_column.name;
column_to_add.type = requested_column.type; column_to_add.type = requested_column.type;

View File

@ -0,0 +1,115 @@
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
#include <DB/Core/Types.h>
#include <DB/IO/ReadHelpers.h>
namespace DB
{
/** Тонкие настройки работы MergeTree.
* Могут быть загружены из конфига.
*/
struct MergeTreeSettings
{
/** Настройки слияний. */
/// Опеределяет, насколько разбалансированные объединения мы готовы делать.
/// Чем больше, тем более разбалансированные. Желательно, чтобы было больше, чем 1 / max_parts_to_merge_at_once.
double size_ratio_coefficient_to_merge_parts = 0.25;
/// Сколько за раз сливать кусков.
/// Трудоемкость выбора кусков O(N * max_parts_to_merge_at_once).
size_t max_parts_to_merge_at_once = 10;
/// Но пока суммарный размер кусков слишком маленький (меньше такого количества байт), можно сливать и больше кусков за раз.
/// Это сделано, чтобы быстрее сливать очень уж маленькие куски, которых может быстро накопиться много.
size_t merge_more_parts_if_sum_bytes_is_less_than = 100 * 1024 * 1024;
size_t max_parts_to_merge_at_once_if_small = 100;
/// Куски настолько большого размера объединять нельзя вообще.
size_t max_bytes_to_merge_parts = 10ul * 1024 * 1024 * 1024;
/// Не больше половины потоков одновременно могут выполнять слияния, в которых участвует хоть один кусок хотя бы такого размера.
size_t max_bytes_to_merge_parts_small = 250 * 1024 * 1024;
/// Куски настолько большого размера в сумме, объединять нельзя вообще.
size_t max_sum_bytes_to_merge_parts = 25ul * 1024 * 1024 * 1024;
/// Во столько раз ночью увеличиваем коэффициент.
size_t merge_parts_at_night_inc = 10;
/// Сколько заданий на слияние кусков разрешено одновременно иметь в очереди ReplicatedMergeTree.
size_t max_replicated_merges_in_queue = 6;
/// Через сколько секунд удалять ненужные куски.
time_t old_parts_lifetime = 8 * 60;
/** Настройки вставок. */
/// Если в таблице хотя бы столько активных кусков, искусственно замедлять вставки в таблицу.
size_t parts_to_delay_insert = 150;
/// Если в таблице parts_to_delay_insert + k кусков, спать insert_delay_step^k миллисекунд перед вставкой каждого блока.
/// Таким образом, скорость вставок автоматически замедлится примерно до скорости слияний.
double insert_delay_step = 1.1;
/** Настройки репликации. */
/// Для скольки последних блоков хранить хеши в ZooKeeper.
size_t replicated_deduplication_window = 100;
/// Хранить примерно столько последних записей в логе в ZooKeeper, даже если они никому уже не нужны.
/// Не влияет на работу таблиц; используется только чтобы успеть посмотреть на лог в ZooKeeper глазами прежде, чем его очистят.
size_t replicated_logs_to_keep = 100;
/// Настройки минимального количества битых данных, при котором отказываться автоматически их удалять.
size_t max_suspicious_broken_parts = 5;
/// Максимальное количество ошибок при загрузке кусков, при котором ReplicatedMergeTree соглашается запускаться.
size_t replicated_max_unexpected_parts = 3;
size_t replicated_max_unexpectedly_merged_parts = 2;
size_t replicated_max_missing_obsolete_parts = 5;
size_t replicated_max_missing_active_parts = 20;
/// Если отношение количества ошибок к общему количеству кусков меньше указанного значения, то всё-равно можно запускаться.
double replicated_max_ratio_of_wrong_parts = 0.05;
void loadFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config)
{
#define SET_DOUBLE(NAME) \
NAME = config.getDouble(config_elem + "." #NAME, NAME);
#define SET_SIZE_T(NAME) \
if (config.has(config_elem + "." #NAME)) NAME = parse<size_t>(config.getString(config_elem + "." #NAME));
SET_DOUBLE(size_ratio_coefficient_to_merge_parts);
SET_SIZE_T(max_parts_to_merge_at_once);
SET_SIZE_T(merge_more_parts_if_sum_bytes_is_less_than);
SET_SIZE_T(max_parts_to_merge_at_once_if_small);
SET_SIZE_T(max_bytes_to_merge_parts);
SET_SIZE_T(max_bytes_to_merge_parts_small);
SET_SIZE_T(max_sum_bytes_to_merge_parts);
SET_SIZE_T(merge_parts_at_night_inc);
SET_SIZE_T(max_replicated_merges_in_queue);
SET_SIZE_T(old_parts_lifetime);
SET_SIZE_T(parts_to_delay_insert);
SET_DOUBLE(insert_delay_step);
SET_SIZE_T(replicated_deduplication_window);
SET_SIZE_T(replicated_logs_to_keep);
SET_SIZE_T(max_suspicious_broken_parts);
SET_SIZE_T(replicated_max_unexpected_parts);
SET_SIZE_T(replicated_max_unexpectedly_merged_parts);
SET_SIZE_T(replicated_max_missing_obsolete_parts);
SET_SIZE_T(replicated_max_missing_active_parts);
SET_DOUBLE(replicated_max_ratio_of_wrong_parts);
#undef SET_SIZE_T
#undef SET_DOUBLE
}
};
}

View File

@ -37,6 +37,8 @@ class MergeTreeWhereOptimizer
static constexpr auto and_function_name = "and"; static constexpr auto and_function_name = "and";
static constexpr auto equals_function_name = "equals"; static constexpr auto equals_function_name = "equals";
static constexpr auto array_join_function_name = "arrayJoin"; static constexpr auto array_join_function_name = "arrayJoin";
static constexpr auto global_in_function_name = "globalIn";
static constexpr auto global_not_in_function_name = "globalNotIn";
public: public:
MergeTreeWhereOptimizer(const MergeTreeWhereOptimizer&) = delete; MergeTreeWhereOptimizer(const MergeTreeWhereOptimizer&) = delete;
@ -312,8 +314,11 @@ private:
} }
/** ARRAY JOIN'ed columns as well as arrayJoin() result cannot be used in PREWHERE, therefore expressions /** ARRAY JOIN'ed columns as well as arrayJoin() result cannot be used in PREWHERE, therefore expressions
* containing said columns should not be moved to PREWHERE at all. * containing said columns should not be moved to PREWHERE at all.
* We assume all AS aliases have been expanded prior to using this class */ * We assume all AS aliases have been expanded prior to using this class
*
* Also, disallow moving expressions with GLOBAL [NOT] IN.
*/
bool cannotBeMoved(const IAST * ptr) const bool cannotBeMoved(const IAST * ptr) const
{ {
if (const auto function_ptr = typeid_cast<const ASTFunction *>(ptr)) if (const auto function_ptr = typeid_cast<const ASTFunction *>(ptr))
@ -321,6 +326,11 @@ private:
/// disallow arrayJoin expressions to be moved to PREWHERE for now /// disallow arrayJoin expressions to be moved to PREWHERE for now
if (array_join_function_name == function_ptr->name) if (array_join_function_name == function_ptr->name)
return true; return true;
/// disallow GLOBAL IN, GLOBAL NOT IN
if (global_in_function_name == function_ptr->name
|| global_not_in_function_name == function_ptr->name)
return true;
} }
else if (const auto identifier_ptr = typeid_cast<const ASTIdentifier *>(ptr)) else if (const auto identifier_ptr = typeid_cast<const ASTIdentifier *>(ptr))
{ {

View File

@ -283,7 +283,7 @@ public:
size_t rows = block.rows(); size_t rows = block.rows();
/// Сначала пишем индекс. Индекс содержит значение Primary Key для каждой index_granularity строки. /// Сначала пишем индекс. Индекс содержит значение Primary Key для каждой index_granularity строки.
typedef std::vector<const ColumnWithNameAndType *> PrimaryColumns; typedef std::vector<const ColumnWithTypeAndName *> PrimaryColumns;
PrimaryColumns primary_columns; PrimaryColumns primary_columns;
for (const auto & descr : storage.getSortDescription()) for (const auto & descr : storage.getSortDescription())
@ -311,7 +311,7 @@ public:
/// Теперь пишем данные. /// Теперь пишем данные.
for (const auto & it : columns_list) for (const auto & it : columns_list)
{ {
const ColumnWithNameAndType & column = block.getByName(it.name); const ColumnWithTypeAndName & column = block.getByName(it.name);
writeData(column.name, *column.type, *column.column, offset_columns); writeData(column.name, *column.type, *column.column, offset_columns);
} }
@ -435,7 +435,7 @@ public:
OffsetColumns offset_columns; OffsetColumns offset_columns;
for (size_t i = 0; i < block.columns(); ++i) for (size_t i = 0; i < block.columns(); ++i)
{ {
const ColumnWithNameAndType & column = block.getByPosition(i); const ColumnWithTypeAndName & column = block.getByPosition(i);
writeData(column.name, *column.type, *column.column, offset_columns); writeData(column.name, *column.type, *column.column, offset_columns);
} }

View File

@ -17,10 +17,10 @@ public:
std::string getName() const override { return "ChunkRef"; } std::string getName() const override { return "ChunkRef"; }
std::string getTableName() const override { return name; } std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsListImpl() const override { return getSource().getColumnsListImpl(); } const NamesAndTypesList & getColumnsListImpl() const override { return typeid_cast<const StorageChunks &>(*getSource()).getColumnsListImpl(); }
/// В таблице, на которую мы ссылаемся, могут быть виртуальные столбцы. /// В таблице, на которую мы ссылаемся, могут быть виртуальные столбцы.
NameAndTypePair getColumn(const String & column_name) const override { return getSource().getColumn(column_name); }; NameAndTypePair getColumn(const String & column_name) const override { return getSource()->getColumn(column_name); };
bool hasColumn(const String & column_name) const override { return getSource().hasColumn(column_name); }; bool hasColumn(const String & column_name) const override { return getSource()->hasColumn(column_name); };
BlockInputStreams read( BlockInputStreams read(
const Names & column_names, const Names & column_names,
@ -47,8 +47,8 @@ private:
StorageChunkRef(const std::string & name_, const Context & context_, const std::string & source_database_name_, const std::string & source_table_name_, bool attach); StorageChunkRef(const std::string & name_, const Context & context_, const std::string & source_database_name_, const std::string & source_table_name_, bool attach);
/// TODO: может быть, можно просто хранить указатель на родительскую таблицу? /// TODO: может быть, можно просто хранить указатель на родительскую таблицу?
StorageChunks & getSource(); StoragePtr getSource();
const StorageChunks & getSource() const; const StoragePtr getSource() const;
}; };
} }

View File

@ -2,7 +2,6 @@
#include <DB/Storages/StorageLog.h> #include <DB/Storages/StorageLog.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
#include <statdaemons/CounterInFile.h>
namespace DB namespace DB
@ -89,7 +88,9 @@ private:
ChunkIndices chunk_indices; ChunkIndices chunk_indices;
ChunkNumToChunkName chunk_names; ChunkNumToChunkName chunk_names;
CounterInFile reference_counter; size_t refcount = 0;
std::mutex refcount_mutex;
Context & context; Context & context;
Logger * log; Logger * log;

View File

@ -43,7 +43,7 @@ public:
MergeTreeData::Mode mode_, MergeTreeData::Mode mode_,
const String & sign_column_, /// Для Collapsing режима. const String & sign_column_, /// Для Collapsing режима.
const Names & columns_to_sum_, /// Для Summing режима. const Names & columns_to_sum_, /// Для Summing режима.
const MergeTreeSettings & settings_ = MergeTreeSettings()); const MergeTreeSettings & settings_);
void shutdown() override; void shutdown() override;
~StorageMergeTree() override; ~StorageMergeTree() override;

View File

@ -43,7 +43,7 @@ public:
MergeTreeData::Mode mode_, MergeTreeData::Mode mode_,
const String & sign_column_, /// Для Collapsing режима. const String & sign_column_, /// Для Collapsing режима.
const Names & columns_to_sum_, /// Для Summing режима. const Names & columns_to_sum_, /// Для Summing режима.
const MergeTreeSettings & settings_ = MergeTreeSettings()); const MergeTreeSettings & settings_);
void shutdown() override; void shutdown() override;
~StorageReplicatedMergeTree() override; ~StorageReplicatedMergeTree() override;
@ -124,6 +124,8 @@ public:
UInt32 inserts_in_queue; UInt32 inserts_in_queue;
UInt32 merges_in_queue; UInt32 merges_in_queue;
UInt32 queue_oldest_time; UInt32 queue_oldest_time;
UInt32 inserts_oldest_time;
UInt32 merges_oldest_time;
UInt64 log_max_index; UInt64 log_max_index;
UInt64 log_pointer; UInt64 log_pointer;
UInt8 total_replicas; UInt8 total_replicas;
@ -282,7 +284,7 @@ private:
MergeTreeData::Mode mode_, MergeTreeData::Mode mode_,
const String & sign_column_, const String & sign_column_,
const Names & columns_to_sum_, const Names & columns_to_sum_,
const MergeTreeSettings & settings_ = MergeTreeSettings()); const MergeTreeSettings & settings_);
/// Инициализация. /// Инициализация.

View File

@ -351,6 +351,26 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
else else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
} }
else if (name == "uniqCombined")
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqCombinedData>(*argument_types[0]);
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqCombinedData<DataTypeDate::FieldType>>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqCombinedData<DataTypeDateTime::FieldType>>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedData<String>>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
else if (name == "uniqUpTo") else if (name == "uniqUpTo")
{ {
if (argument_types.size() != 1) if (argument_types.size() != 1)
@ -706,6 +726,7 @@ const AggregateFunctionFactory::FunctionNames & AggregateFunctionFactory::getFun
"uniq", "uniq",
"uniqHLL12", "uniqHLL12",
"uniqExact", "uniqExact",
"uniqCombined",
"uniqUpTo", "uniqUpTo",
"groupArray", "groupArray",
"groupUniqArray", "groupUniqArray",

View File

@ -152,7 +152,7 @@ bool filterBlockWithQuery(ASTPtr query, Block & block, const Context & context)
return false; return false;
/// Распарсим и вычислим выражение. /// Распарсим и вычислим выражение.
ExpressionAnalyzer analyzer(expression_ast, context, block.getColumnsList()); ExpressionAnalyzer analyzer(expression_ast, context, {}, block.getColumnsList());
ExpressionActionsPtr actions = analyzer.getActions(false); ExpressionActionsPtr actions = analyzer.getActions(false);
actions->execute(block); actions->execute(block);

View File

@ -51,7 +51,7 @@ void Block::addDefaults(const NamesAndTypesList & required_columns)
if (has(requested_column.name)) if (has(requested_column.name))
continue; continue;
ColumnWithNameAndType column_to_add; ColumnWithTypeAndName column_to_add;
column_to_add.name = requested_column.name; column_to_add.name = requested_column.name;
column_to_add.type = requested_column.type; column_to_add.type = requested_column.type;
@ -101,7 +101,7 @@ Block & Block::operator= (const Block & other)
return *this; return *this;
} }
void Block::insert(size_t position, const ColumnWithNameAndType & elem) void Block::insert(size_t position, const ColumnWithTypeAndName & elem)
{ {
if (position > index_by_position.size()) if (position > index_by_position.size())
throw Exception("Position out of bound in Block::insert(), max position = " throw Exception("Position out of bound in Block::insert(), max position = "
@ -124,7 +124,7 @@ void Block::insert(size_t position, const ColumnWithNameAndType & elem)
} }
void Block::insert(const ColumnWithNameAndType & elem) void Block::insert(const ColumnWithTypeAndName & elem)
{ {
Container_t::iterator it = data.insert(data.end(), elem); Container_t::iterator it = data.insert(data.end(), elem);
index_by_name[elem.name] = it; index_by_name[elem.name] = it;
@ -132,7 +132,7 @@ void Block::insert(const ColumnWithNameAndType & elem)
} }
void Block::insertUnique(const ColumnWithNameAndType & elem) void Block::insertUnique(const ColumnWithTypeAndName & elem)
{ {
if (index_by_name.end() == index_by_name.find(elem.name)) if (index_by_name.end() == index_by_name.find(elem.name))
insert(elem); insert(elem);
@ -175,7 +175,7 @@ void Block::erase(const String & name)
} }
ColumnWithNameAndType & Block::getByPosition(size_t position) ColumnWithTypeAndName & Block::getByPosition(size_t position)
{ {
if (position >= index_by_position.size()) if (position >= index_by_position.size())
throw Exception("Position " + toString(position) throw Exception("Position " + toString(position)
@ -187,7 +187,7 @@ ColumnWithNameAndType & Block::getByPosition(size_t position)
} }
const ColumnWithNameAndType & Block::getByPosition(size_t position) const const ColumnWithTypeAndName & Block::getByPosition(size_t position) const
{ {
if (position >= index_by_position.size()) if (position >= index_by_position.size())
throw Exception("Position " + toString(position) throw Exception("Position " + toString(position)
@ -199,7 +199,7 @@ const ColumnWithNameAndType & Block::getByPosition(size_t position) const
} }
ColumnWithNameAndType & Block::getByName(const std::string & name) ColumnWithTypeAndName & Block::getByName(const std::string & name)
{ {
IndexByName_t::const_iterator it = index_by_name.find(name); IndexByName_t::const_iterator it = index_by_name.find(name);
if (index_by_name.end() == it) if (index_by_name.end() == it)
@ -210,7 +210,7 @@ ColumnWithNameAndType & Block::getByName(const std::string & name)
} }
const ColumnWithNameAndType & Block::getByName(const std::string & name) const const ColumnWithTypeAndName & Block::getByName(const std::string & name) const
{ {
IndexByName_t::const_iterator it = index_by_name.find(name); IndexByName_t::const_iterator it = index_by_name.find(name);
if (index_by_name.end() == it) if (index_by_name.end() == it)
@ -330,9 +330,9 @@ Block Block::sortColumns() const
} }
ColumnsWithNameAndType Block::getColumns() const ColumnsWithTypeAndName Block::getColumns() const
{ {
return ColumnsWithNameAndType(data.begin(), data.end()); return ColumnsWithTypeAndName(data.begin(), data.end());
} }

View File

@ -37,7 +37,7 @@ Block AggregatingSortedBlockInputStream::readImpl()
/// Заполним номера столбцов, которые нужно доагрегировать. /// Заполним номера столбцов, которые нужно доагрегировать.
for (size_t i = 0; i < num_columns; ++i) for (size_t i = 0; i < num_columns; ++i)
{ {
ColumnWithNameAndType & column = merged_block.getByPosition(i); ColumnWithTypeAndName & column = merged_block.getByPosition(i);
/// Оставляем только состояния аггрегатных функций. /// Оставляем только состояния аггрегатных функций.
if (strncmp(column.type->getName().data(), "AggregateFunction", strlen("AggregateFunction")) != 0) if (strncmp(column.type->getName().data(), "AggregateFunction", strlen("AggregateFunction")) != 0)

View File

@ -75,7 +75,7 @@ Block FilterBlockInputStream::readImpl()
size_t filtered_rows = 0; size_t filtered_rows = 0;
if (first_non_constant_column != static_cast<size_t>(filter_column)) if (first_non_constant_column != static_cast<size_t>(filter_column))
{ {
ColumnWithNameAndType & current_column = res.getByPosition(first_non_constant_column); ColumnWithTypeAndName & current_column = res.getByPosition(first_non_constant_column);
current_column.column = current_column.column->filter(filter); current_column.column = current_column.column->filter(filter);
filtered_rows = current_column.column->size(); filtered_rows = current_column.column->size();
} }
@ -100,7 +100,7 @@ Block FilterBlockInputStream::readImpl()
/// Фильтруем остальные столбцы. /// Фильтруем остальные столбцы.
for (size_t i = 0; i < columns; ++i) for (size_t i = 0; i < columns; ++i)
{ {
ColumnWithNameAndType & current_column = res.getByPosition(i); ColumnWithTypeAndName & current_column = res.getByPosition(i);
if (i == static_cast<size_t>(filter_column)) if (i == static_cast<size_t>(filter_column))
{ {

View File

@ -17,6 +17,7 @@
#include <DB/DataStreams/BlockOutputStreamFromRowOutputStream.h> #include <DB/DataStreams/BlockOutputStreamFromRowOutputStream.h>
#include <DB/DataStreams/JSONRowOutputStream.h> #include <DB/DataStreams/JSONRowOutputStream.h>
#include <DB/DataStreams/JSONCompactRowOutputStream.h> #include <DB/DataStreams/JSONCompactRowOutputStream.h>
#include <DB/DataStreams/TSKVRowOutputStream.h>
#include <DB/DataStreams/PrettyCompactMonoBlockOutputStream.h> #include <DB/DataStreams/PrettyCompactMonoBlockOutputStream.h>
#include <DB/DataStreams/FormatFactory.h> #include <DB/DataStreams/FormatFactory.h>
@ -83,6 +84,8 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer &
return new BlockOutputStreamFromRowOutputStream(new JSONRowOutputStream(buf, sample)); return new BlockOutputStreamFromRowOutputStream(new JSONRowOutputStream(buf, sample));
else if (name == "JSONCompact") else if (name == "JSONCompact")
return new BlockOutputStreamFromRowOutputStream(new JSONCompactRowOutputStream(buf, sample)); return new BlockOutputStreamFromRowOutputStream(new JSONCompactRowOutputStream(buf, sample));
else if (name == "TSKV")
return new BlockOutputStreamFromRowOutputStream(new TSKVRowOutputStream(buf, sample));
else if (name == "Null") else if (name == "Null")
return new NullBlockOutputStream; return new NullBlockOutputStream;
else if (name == "PrettyCompactMonoBlock") else if (name == "PrettyCompactMonoBlock")

View File

@ -58,7 +58,7 @@ void JSONCompactRowOutputStream::writeTotals()
if (i != 0) if (i != 0)
writeChar(',', *ostr); writeChar(',', *ostr);
const ColumnWithNameAndType & column = totals.getByPosition(i); const ColumnWithTypeAndName & column = totals.getByPosition(i);
column.type->serializeTextJSON((*column.column)[0], *ostr); column.type->serializeTextJSON((*column.column)[0], *ostr);
} }
@ -79,7 +79,7 @@ static void writeExtremesElement(const char * title, const Block & extremes, siz
if (i != 0) if (i != 0)
writeChar(',', ostr); writeChar(',', ostr);
const ColumnWithNameAndType & column = extremes.getByPosition(i); const ColumnWithTypeAndName & column = extremes.getByPosition(i);
column.type->serializeTextJSON((*column.column)[row_num], ostr); column.type->serializeTextJSON((*column.column)[row_num], ostr);
} }

View File

@ -139,7 +139,7 @@ void JSONRowOutputStream::writeTotals()
size_t totals_columns = totals.columns(); size_t totals_columns = totals.columns();
for (size_t i = 0; i < totals_columns; ++i) for (size_t i = 0; i < totals_columns; ++i)
{ {
const ColumnWithNameAndType & column = totals.getByPosition(i); const ColumnWithTypeAndName & column = totals.getByPosition(i);
if (i != 0) if (i != 0)
writeCString(",\n", *ostr); writeCString(",\n", *ostr);
@ -166,7 +166,7 @@ static void writeExtremesElement(const char * title, const Block & extremes, siz
size_t extremes_columns = extremes.columns(); size_t extremes_columns = extremes.columns();
for (size_t i = 0; i < extremes_columns; ++i) for (size_t i = 0; i < extremes_columns; ++i)
{ {
const ColumnWithNameAndType & column = extremes.getByPosition(i); const ColumnWithTypeAndName & column = extremes.getByPosition(i);
if (i != 0) if (i != 0)
writeCString(",\n", ostr); writeCString(",\n", ostr);

View File

@ -62,7 +62,7 @@ Block NativeBlockInputStream::readImpl()
for (size_t i = 0; i < columns; ++i) for (size_t i = 0; i < columns; ++i)
{ {
ColumnWithNameAndType column; ColumnWithTypeAndName column;
/// Имя /// Имя
readStringBinary(column.name, istr); readStringBinary(column.name, istr);

View File

@ -76,7 +76,7 @@ void NativeBlockOutputStream::write(const Block & block)
for (size_t i = 0; i < columns; ++i) for (size_t i = 0; i < columns; ++i)
{ {
const ColumnWithNameAndType & column = block.getByPosition(i); const ColumnWithTypeAndName & column = block.getByPosition(i);
/// Имя /// Имя
writeStringBinary(column.name, ostr); writeStringBinary(column.name, ostr);

View File

@ -32,7 +32,7 @@ void PrettyBlockOutputStream::calculateWidths(Block & block, Widths_t & max_widt
/// Вычислим ширину всех значений /// Вычислим ширину всех значений
for (size_t i = 0; i < columns; ++i) for (size_t i = 0; i < columns; ++i)
{ {
ColumnWithNameAndType column; ColumnWithTypeAndName column;
column.type = visible_width_type; column.type = visible_width_type;
column.name = "visibleWidth(" + block.getByPosition(i).name + ")"; column.name = "visibleWidth(" + block.getByPosition(i).name + ")";
@ -137,7 +137,7 @@ void PrettyBlockOutputStream::write(const Block & block_)
if (i != 0) if (i != 0)
writeCString("", ostr); writeCString("", ostr);
const ColumnWithNameAndType & col = block.getByPosition(i); const ColumnWithTypeAndName & col = block.getByPosition(i);
if (!no_escapes) if (!no_escapes)
writeCString("\033[1m", ostr); writeCString("\033[1m", ostr);
@ -176,7 +176,7 @@ void PrettyBlockOutputStream::write(const Block & block_)
if (j != 0) if (j != 0)
writeCString("", ostr); writeCString("", ostr);
const ColumnWithNameAndType & col = block.getByPosition(j); const ColumnWithTypeAndName & col = block.getByPosition(j);
if (col.type->isNumeric()) if (col.type->isNumeric())
{ {

View File

@ -21,7 +21,7 @@ void PrettyCompactBlockOutputStream::writeHeader(
if (i != 0) if (i != 0)
writeCString("─┬─", ostr); writeCString("─┬─", ostr);
const ColumnWithNameAndType & col = block.getByPosition(i); const ColumnWithTypeAndName & col = block.getByPosition(i);
if (col.type->isNumeric()) if (col.type->isNumeric())
{ {
@ -83,7 +83,7 @@ void PrettyCompactBlockOutputStream::writeRow(
if (j != 0) if (j != 0)
writeCString("", ostr); writeCString("", ostr);
const ColumnWithNameAndType & col = block.getByPosition(j); const ColumnWithTypeAndName & col = block.getByPosition(j);
if (col.type->isNumeric()) if (col.type->isNumeric())
{ {

View File

@ -40,7 +40,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
if (i != 0) if (i != 0)
writeCString(" ", ostr); writeCString(" ", ostr);
const ColumnWithNameAndType & col = block.getByPosition(i); const ColumnWithTypeAndName & col = block.getByPosition(i);
if (col.type->isNumeric()) if (col.type->isNumeric())
{ {
@ -74,7 +74,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
if (j != 0) if (j != 0)
writeCString(" ", ostr); writeCString(" ", ostr);
const ColumnWithNameAndType & col = block.getByPosition(j); const ColumnWithTypeAndName & col = block.getByPosition(j);
if (col.type->isNumeric()) if (col.type->isNumeric())
{ {

View File

@ -63,7 +63,7 @@ Block SummingSortedBlockInputStream::readImpl()
*/ */
for (size_t i = 0; i < num_columns; ++i) for (size_t i = 0; i < num_columns; ++i)
{ {
ColumnWithNameAndType & column = merged_block.getByPosition(i); ColumnWithTypeAndName & column = merged_block.getByPosition(i);
/// Discover nested Maps and find columns for summation /// Discover nested Maps and find columns for summation
if (const auto array_type = typeid_cast<const DataTypeArray *>(column.type.get())) if (const auto array_type = typeid_cast<const DataTypeArray *>(column.type.get()))

View File

@ -0,0 +1,37 @@
#include <DB/IO/WriteHelpers.h>
#include <DB/DataStreams/TSKVRowOutputStream.h>
namespace DB
{
using Poco::SharedPtr;
TSKVRowOutputStream::TSKVRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
: TabSeparatedRowOutputStream(ostr_, sample_)
{
NamesAndTypesList columns(sample_.getColumnsList());
fields.assign(columns.begin(), columns.end());
for (auto & field : fields)
{
String escaped_field_name;
{
WriteBufferFromString wb(escaped_field_name);
writeAnyEscapedString<'='>(field.name.data(), field.name.data() + field.name.size(), wb);
}
field.name = escaped_field_name;
}
}
void TSKVRowOutputStream::writeField(const Field & field)
{
writeString(fields[field_number].name, ostr);
writeCString("=", ostr);
data_types[field_number]->serializeTextEscaped(field, ostr);
++field_number;
}
}

View File

@ -9,7 +9,7 @@ void TabSeparatedBlockOutputStream::write(const Block & block)
size_t columns = block.columns(); size_t columns = block.columns();
for (size_t i = 0; i < columns; ++i) for (size_t i = 0; i < columns; ++i)
{ {
const ColumnWithNameAndType & col = block.getByPosition(i); const ColumnWithTypeAndName & col = block.getByPosition(i);
size_t rows = block.rows(); size_t rows = block.rows();
for (size_t j = 0; j < rows; ++j) for (size_t j = 0; j < rows; ++j)

View File

@ -9,7 +9,7 @@ static void finalize(Block & block)
{ {
for (size_t i = 0; i < block.columns(); ++i) for (size_t i = 0; i < block.columns(); ++i)
{ {
ColumnWithNameAndType & current = block.getByPosition(i); ColumnWithTypeAndName & current = block.getByPosition(i);
ColumnAggregateFunction * unfinalized_column = typeid_cast<ColumnAggregateFunction *>(&*current.column); ColumnAggregateFunction * unfinalized_column = typeid_cast<ColumnAggregateFunction *>(&*current.column);
if (unfinalized_column) if (unfinalized_column)
{ {
@ -106,7 +106,7 @@ Block TotalsHavingBlockInputStream::readImpl()
for (size_t i = 0; i < columns; ++i) for (size_t i = 0; i < columns; ++i)
{ {
ColumnWithNameAndType & current_column = finalized.getByPosition(i); ColumnWithTypeAndName & current_column = finalized.getByPosition(i);
current_column.column = current_column.column->filter(filter); current_column.column = current_column.column->filter(filter);
if (current_column.column->empty()) if (current_column.column->empty())
{ {
@ -134,7 +134,7 @@ void TotalsHavingBlockInputStream::addToTotals(Block & totals, Block & block, co
for (size_t i = 0; i < block.columns(); ++i) for (size_t i = 0; i < block.columns(); ++i)
{ {
const ColumnWithNameAndType & current = block.getByPosition(i); const ColumnWithTypeAndName & current = block.getByPosition(i);
const ColumnAggregateFunction * column = typeid_cast<const ColumnAggregateFunction *>(&*current.column); const ColumnAggregateFunction * column = typeid_cast<const ColumnAggregateFunction *>(&*current.column);
if (!column) if (!column)
@ -143,7 +143,7 @@ void TotalsHavingBlockInputStream::addToTotals(Block & totals, Block & block, co
{ {
ColumnPtr new_column = current.type->createColumn(); ColumnPtr new_column = current.type->createColumn();
new_column->insertDefault(); new_column->insertDefault();
totals.insert(ColumnWithNameAndType(new_column, current.type, current.name)); totals.insert(ColumnWithTypeAndName(new_column, current.type, current.name));
} }
continue; continue;
} }
@ -156,7 +156,7 @@ void TotalsHavingBlockInputStream::addToTotals(Block & totals, Block & block, co
{ {
function = column->getAggregateFunction(); function = column->getAggregateFunction();
target = new ColumnAggregateFunction(column->getAggregateFunction(), Arenas(1, arena)); target = new ColumnAggregateFunction(column->getAggregateFunction(), Arenas(1, arena));
totals.insert(ColumnWithNameAndType(target, current.type, current.name)); totals.insert(ColumnWithTypeAndName(target, current.type, current.name));
data = arena->alloc(function->sizeOfData()); data = arena->alloc(function->sizeOfData());
function->create(data); function->create(data);

View File

@ -31,7 +31,7 @@ int main(int argc, char ** argv)
Block block; Block block;
ColumnWithNameAndType column_x; ColumnWithTypeAndName column_x;
column_x.name = "x"; column_x.name = "x";
column_x.type = new DataTypeInt16; column_x.type = new DataTypeInt16;
ColumnInt16 * x = new ColumnInt16; ColumnInt16 * x = new ColumnInt16;
@ -46,7 +46,7 @@ int main(int argc, char ** argv)
const char * strings[] = {"abc", "def", "abcd", "defg", "ac"}; const char * strings[] = {"abc", "def", "abcd", "defg", "ac"};
ColumnWithNameAndType column_s1; ColumnWithTypeAndName column_s1;
column_s1.name = "s1"; column_s1.name = "s1";
column_s1.type = new DataTypeString; column_s1.type = new DataTypeString;
column_s1.column = new ColumnString; column_s1.column = new ColumnString;
@ -56,7 +56,7 @@ int main(int argc, char ** argv)
block.insert(column_s1); block.insert(column_s1);
ColumnWithNameAndType column_s2; ColumnWithTypeAndName column_s2;
column_s2.name = "s2"; column_s2.name = "s2";
column_s2.type = new DataTypeString; column_s2.type = new DataTypeString;
column_s2.column = new ColumnString; column_s2.column = new ColumnString;
@ -86,7 +86,7 @@ int main(int argc, char ** argv)
Block sample; Block sample;
for (DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it) for (DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it)
{ {
ColumnWithNameAndType col; ColumnWithTypeAndName col;
col.type = *it; col.type = *it;
sample.insert(col); sample.insert(col);
} }

View File

@ -7,7 +7,7 @@
#include <Poco/SharedPtr.h> #include <Poco/SharedPtr.h>
#include <DB/Core/Block.h> #include <DB/Core/Block.h>
#include <DB/Core/ColumnWithNameAndType.h> #include <DB/Core/ColumnWithTypeAndName.h>
#include <DB/IO/ReadBufferFromIStream.h> #include <DB/IO/ReadBufferFromIStream.h>
#include <DB/IO/WriteBufferFromOStream.h> #include <DB/IO/WriteBufferFromOStream.h>
@ -28,13 +28,13 @@ int main(int argc, char ** argv)
{ {
DB::Block sample; DB::Block sample;
DB::ColumnWithNameAndType col1; DB::ColumnWithTypeAndName col1;
col1.name = "col1"; col1.name = "col1";
col1.type = new DB::DataTypeUInt64; col1.type = new DB::DataTypeUInt64;
col1.column = col1.type->createColumn(); col1.column = col1.type->createColumn();
sample.insert(col1); sample.insert(col1);
DB::ColumnWithNameAndType col2; DB::ColumnWithTypeAndName col2;
col2.name = "col2"; col2.name = "col2";
col2.type = new DB::DataTypeString; col2.type = new DB::DataTypeString;
col2.column = col2.type->createColumn(); col2.column = col2.type->createColumn();

View File

@ -98,7 +98,7 @@ int main(int argc, char ** argv)
Block sample; Block sample;
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it) for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
{ {
ColumnWithNameAndType elem; ColumnWithTypeAndName elem;
elem.name = it->name; elem.name = it->name;
elem.type = it->type; elem.type = it->type;
elem.column = elem.type->createColumn(); elem.column = elem.type->createColumn();

View File

@ -32,7 +32,7 @@ int main(int argc, char ** argv)
Block block1; Block block1;
{ {
ColumnWithNameAndType column1; ColumnWithTypeAndName column1;
column1.name = "Sign"; column1.name = "Sign";
column1.type = new DataTypeInt8; column1.type = new DataTypeInt8;
column1.column = new ColumnInt8; column1.column = new ColumnInt8;
@ -40,7 +40,7 @@ int main(int argc, char ** argv)
column1.column->insert(DB::Int64(-1)); column1.column->insert(DB::Int64(-1));
block1.insert(column1); block1.insert(column1);
ColumnWithNameAndType column2; ColumnWithTypeAndName column2;
column2.name = "CounterID"; column2.name = "CounterID";
column2.type = new DataTypeUInt32; column2.type = new DataTypeUInt32;
column2.column = new ColumnUInt32; column2.column = new ColumnUInt32;
@ -52,7 +52,7 @@ int main(int argc, char ** argv)
Block block2; Block block2;
{ {
ColumnWithNameAndType column1; ColumnWithTypeAndName column1;
column1.name = "Sign"; column1.name = "Sign";
column1.type = new DataTypeInt8; column1.type = new DataTypeInt8;
column1.column = new ColumnInt8; column1.column = new ColumnInt8;
@ -60,7 +60,7 @@ int main(int argc, char ** argv)
column1.column->insert(DB::Int64(1)); column1.column->insert(DB::Int64(1));
block2.insert(column1); block2.insert(column1);
ColumnWithNameAndType column2; ColumnWithTypeAndName column2;
column2.name = "CounterID"; column2.name = "CounterID";
column2.type = new DataTypeUInt32; column2.type = new DataTypeUInt32;
column2.column = new ColumnUInt32; column2.column = new ColumnUInt32;

View File

@ -38,9 +38,8 @@ int main(int argc, char ** argv)
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), ""); ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
Context context; Context context;
context.getColumns().push_back(NameAndTypePair("number", new DataTypeUInt64));
ExpressionAnalyzer analyzer(ast, context, context.getColumns()); ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", new DataTypeUInt64)});
ExpressionActionsChain chain; ExpressionActionsChain chain;
analyzer.appendSelect(chain, false); analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain, false); analyzer.appendProjectResult(chain, false);

View File

@ -44,9 +44,8 @@ int main(int argc, char ** argv)
std::cerr << ast->getTreeID() << std::endl; std::cerr << ast->getTreeID() << std::endl;
Context context; Context context;
context.getColumns().push_back(NameAndTypePair("number", new DataTypeUInt64));
ExpressionAnalyzer analyzer(ast, context, context.getColumns()); ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", new DataTypeUInt64)});
ExpressionActionsChain chain; ExpressionActionsChain chain;
analyzer.appendSelect(chain, false); analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain, false); analyzer.appendProjectResult(chain, false);

View File

@ -98,8 +98,6 @@ int main(int argc, char ** argv)
Context context; Context context;
context.getColumns() = *names_and_types_list;
std::string input = "SELECT UniqID, URL, CounterID, IsLink WHERE URL = 'http://mail.yandex.ru/neo2/#inbox'"; std::string input = "SELECT UniqID, URL, CounterID, IsLink WHERE URL = 'http://mail.yandex.ru/neo2/#inbox'";
ParserSelectQuery parser; ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), ""); ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
@ -114,7 +112,7 @@ int main(int argc, char ** argv)
/// читаем из неё, применяем выражение, фильтруем, и пишем в tsv виде в консоль /// читаем из неё, применяем выражение, фильтруем, и пишем в tsv виде в консоль
ExpressionAnalyzer analyzer(ast, context, context.getColumns()); ExpressionAnalyzer analyzer(ast, context, nullptr, *names_and_types_list);
ExpressionActionsChain chain; ExpressionActionsChain chain;
analyzer.appendSelect(chain, false); analyzer.appendSelect(chain, false);
analyzer.appendWhere(chain, false); analyzer.appendWhere(chain, false);

View File

@ -62,9 +62,8 @@ int main(int argc, char ** argv)
std::cerr << std::endl; std::cerr << std::endl;
Context context; Context context;
context.getColumns().push_back(NameAndTypePair("number", new DataTypeUInt64));
ExpressionAnalyzer analyzer(ast, context, context.getColumns()); ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", new DataTypeUInt64)});
ExpressionActionsChain chain; ExpressionActionsChain chain;
analyzer.appendSelect(chain, false); analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain, false); analyzer.appendProjectResult(chain, false);

View File

@ -51,7 +51,7 @@ int main(int argc, char ** argv)
Block sample; Block sample;
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it) for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
{ {
ColumnWithNameAndType elem; ColumnWithTypeAndName elem;
elem.name = it->name; elem.name = it->name;
elem.type = it->type; elem.type = it->type;
elem.column = elem.type->createColumn(); elem.column = elem.type->createColumn();

View File

@ -136,7 +136,7 @@ int main(int argc, char ** argv)
Block sample; Block sample;
for (DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it) for (DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it)
{ {
ColumnWithNameAndType col; ColumnWithTypeAndName col;
col.type = *it; col.type = *it;
sample.insert(col); sample.insert(col);
} }

View File

@ -21,12 +21,12 @@ int main(int argc, char ** argv)
{ {
DB::Block sample; DB::Block sample;
{ {
DB::ColumnWithNameAndType col; DB::ColumnWithTypeAndName col;
col.type = new DB::DataTypeUInt64; col.type = new DB::DataTypeUInt64;
sample.insert(col); sample.insert(col);
} }
{ {
DB::ColumnWithNameAndType col; DB::ColumnWithTypeAndName col;
col.type = new DB::DataTypeString; col.type = new DB::DataTypeString;
sample.insert(col); sample.insert(col);
} }

View File

@ -38,7 +38,7 @@ int main(int argc, char ** argv)
streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage2, 1)[0], 30, 2000)); streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage2, 1)[0], 30, 2000));
streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage3, 1)[0], 30, 100)); streams.emplace_back(new DB::LimitBlockInputStream(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage3, 1)[0], 30, 100));
DB::UnionBlockInputStream union_stream(streams, 2); DB::UnionBlockInputStream union_stream(streams, nullptr, 2);
DB::FormatFactory format_factory; DB::FormatFactory format_factory;
DB::WriteBufferFromFileDescriptor wb(STDERR_FILENO); DB::WriteBufferFromFileDescriptor wb(STDERR_FILENO);

View File

@ -44,7 +44,7 @@ int main(int argc, char ** argv)
for (size_t i = 0, size = streams.size(); i < size; ++i) for (size_t i = 0, size = streams.size(); i < size; ++i)
streams[i] = new DB::AsynchronousBlockInputStream(streams[i]); streams[i] = new DB::AsynchronousBlockInputStream(streams[i]);
DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream(streams, settings.max_threads); DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream(streams, nullptr, settings.max_threads);
stream = new DB::LimitBlockInputStream(stream, 10); stream = new DB::LimitBlockInputStream(stream, 10);
DB::FormatFactory format_factory; DB::FormatFactory format_factory;

View File

@ -15,6 +15,8 @@ void registerFunctionsHashing(FunctionFactory & factory)
factory.registerFunction<FunctionSipHash64>(); factory.registerFunction<FunctionSipHash64>();
factory.registerFunction<FunctionSipHash128>(); factory.registerFunction<FunctionSipHash128>();
factory.registerFunction<FunctionCityHash64>(); factory.registerFunction<FunctionCityHash64>();
factory.registerFunction<FunctionFarmHash64>();
factory.registerFunction<FunctionMetroHash64>();
factory.registerFunction<FunctionIntHash32>(); factory.registerFunction<FunctionIntHash32>();
factory.registerFunction<FunctionIntHash64>(); factory.registerFunction<FunctionIntHash64>();
factory.registerFunction<FunctionURLHash>(); factory.registerFunction<FunctionURLHash>();

View File

@ -35,14 +35,26 @@ static void numWidthConstant(T a, UInt64 & c)
inline UInt64 floatWidth(const double x) inline UInt64 floatWidth(const double x)
{ {
/// Не быстро.
char tmp[25]; char tmp[25];
double_conversion::StringBuilder builder{tmp, sizeof(tmp)}; double_conversion::StringBuilder builder{tmp, sizeof(tmp)};
const auto result = getDoubleToStringConverter<false>().ToShortest(x, &builder); const auto result = getDoubleToStringConverter<false>().ToShortest(x, &builder);
if (!result) if (!result)
throw Exception("Cannot print float or double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER); throw Exception("Cannot print double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
return builder.position();
}
inline UInt64 floatWidth(const float x)
{
char tmp[25];
double_conversion::StringBuilder builder{tmp, sizeof(tmp)};
const auto result = getDoubleToStringConverter<false>().ToShortestSingle(x, &builder);
if (!result)
throw Exception("Cannot print float number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
return builder.position(); return builder.position();
} }
@ -141,7 +153,7 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
|| VisibleWidth::executeConstNumber<Int16>(block, column, result) || VisibleWidth::executeConstNumber<Int16>(block, column, result)
|| VisibleWidth::executeConstNumber<Int32>(block, column, result) || VisibleWidth::executeConstNumber<Int32>(block, column, result)
|| VisibleWidth::executeConstNumber<Int64>(block, column, result) || VisibleWidth::executeConstNumber<Int64>(block, column, result)
|| VisibleWidth::executeConstNumber<Float32>(block, column, result) /// TODO: правильная работа с float || VisibleWidth::executeConstNumber<Float32>(block, column, result)
|| VisibleWidth::executeConstNumber<Float64>(block, column, result) || VisibleWidth::executeConstNumber<Float64>(block, column, result)
|| VisibleWidth::executeNumber<UInt8>(block, column, result) || VisibleWidth::executeNumber<UInt8>(block, column, result)
|| VisibleWidth::executeNumber<UInt16>(block, column, result) || VisibleWidth::executeNumber<UInt16>(block, column, result)
@ -177,12 +189,12 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
{ {
/// Вычисляем видимую ширину для значений массива. /// Вычисляем видимую ширину для значений массива.
Block nested_block; Block nested_block;
ColumnWithNameAndType nested_values; ColumnWithTypeAndName nested_values;
nested_values.type = typeid_cast<const DataTypeArray &>(*type).getNestedType(); nested_values.type = typeid_cast<const DataTypeArray &>(*type).getNestedType();
nested_values.column = col->getDataPtr(); nested_values.column = col->getDataPtr();
nested_block.insert(nested_values); nested_block.insert(nested_values);
ColumnWithNameAndType nested_result; ColumnWithTypeAndName nested_result;
nested_result.type = new DataTypeUInt64; nested_result.type = new DataTypeUInt64;
nested_block.insert(nested_result); nested_block.insert(nested_result);
@ -241,7 +253,7 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
* x1, x2, x3... , width1, width2, width1 + width2, width3, width1 + width2 + width3, ... * x1, x2, x3... , width1, width2, width1 + width2, width3, width1 + width2 + width3, ...
*/ */
ColumnWithNameAndType nested_result; ColumnWithTypeAndName nested_result;
nested_result.type = new DataTypeUInt64; nested_result.type = new DataTypeUInt64;
nested_block.insert(nested_result); nested_block.insert(nested_result);
@ -250,7 +262,7 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
if (i != 0) if (i != 0)
{ {
ColumnWithNameAndType plus_result; ColumnWithTypeAndName plus_result;
plus_result.type = new DataTypeUInt64; plus_result.type = new DataTypeUInt64;
nested_block.insert(plus_result); nested_block.insert(plus_result);
@ -325,6 +337,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
factory.registerFunction<FunctionSleep>(); factory.registerFunction<FunctionSleep>();
factory.registerFunction<FunctionMaterialize>(); factory.registerFunction<FunctionMaterialize>();
factory.registerFunction<FunctionIgnore>(); factory.registerFunction<FunctionIgnore>();
factory.registerFunction<FunctionIdentity>();
factory.registerFunction<FunctionArrayJoin>(); factory.registerFunction<FunctionArrayJoin>();
factory.registerFunction<FunctionReplicate>(); factory.registerFunction<FunctionReplicate>();
factory.registerFunction<FunctionBar>(); factory.registerFunction<FunctionBar>();

View File

@ -12,14 +12,14 @@ int main(int argc, char ** argv)
{ {
size_t n = atoi(argv[1]); size_t n = atoi(argv[1]);
DB::ColumnWithNameAndType descr1; DB::ColumnWithTypeAndName descr1;
DB::ColumnUInt8 * col1 = new DB::ColumnUInt8; DB::ColumnUInt8 * col1 = new DB::ColumnUInt8;
descr1.type = new DB::DataTypeUInt8; descr1.type = new DB::DataTypeUInt8;
descr1.column = col1; descr1.column = col1;
descr1.name = "x"; descr1.name = "x";
col1->getData().resize(n); col1->getData().resize(n);
DB::ColumnWithNameAndType descr2; DB::ColumnWithTypeAndName descr2;
DB::ColumnInt16 * col2 = new DB::ColumnInt16; DB::ColumnInt16 * col2 = new DB::ColumnInt16;
descr2.type = new DB::DataTypeInt16; descr2.type = new DB::DataTypeInt16;
descr2.column = col2; descr2.column = col2;
@ -49,7 +49,7 @@ int main(int argc, char ** argv)
DB::DataTypePtr res_type = f.getReturnType(arg_types); DB::DataTypePtr res_type = f.getReturnType(arg_types);
DB::ColumnWithNameAndType descr_res; DB::ColumnWithTypeAndName descr_res;
descr_res.type = res_type; descr_res.type = res_type;
descr_res.name = "z"; descr_res.name = "z";

View File

@ -337,7 +337,7 @@ int main(int argc, char ** argv)
for (size_t i = 0; i < columns; ++i) for (size_t i = 0; i < columns; ++i)
{ {
ColumnVector<UInt8> * column = new ColumnVector<UInt8>(block_size); ColumnVector<UInt8> * column = new ColumnVector<UInt8>(block_size);
blocks[b].insert(ColumnWithNameAndType(column, new DataTypeUInt8, "v" + toString(i))); blocks[b].insert(ColumnWithTypeAndName(column, new DataTypeUInt8, "v" + toString(i)));
ColumnVector<UInt8>::Container_t & vec = column->getData(); ColumnVector<UInt8>::Container_t & vec = column->getData();
vec.resize(block_size); vec.resize(block_size);
@ -351,7 +351,7 @@ int main(int argc, char ** argv)
for (size_t b = 0; b < block_count; ++b) for (size_t b = 0; b < block_count; ++b)
{ {
ColumnVector<UInt8> * result_column = new ColumnVector<UInt8>; ColumnVector<UInt8> * result_column = new ColumnVector<UInt8>;
blocks[b].insert(ColumnWithNameAndType(result_column, new DataTypeUInt8, "x")); blocks[b].insert(ColumnWithTypeAndName(result_column, new DataTypeUInt8, "x"));
result_column->getData().resize(block_size); result_column->getData().resize(block_size);
} }

View File

@ -121,7 +121,7 @@ void Aggregator::initialize(Block & block)
for (size_t i = 0; i < aggregates_size; ++i) for (size_t i = 0; i < aggregates_size; ++i)
{ {
ColumnWithNameAndType col; ColumnWithTypeAndName col;
col.name = aggregates[i].column_name; col.name = aggregates[i].column_name;
size_t arguments_size = aggregates[i].arguments.size(); size_t arguments_size = aggregates[i].arguments.size();
@ -848,7 +848,7 @@ Block Aggregator::prepareBlockAndFill(
} }
else else
{ {
ColumnWithNameAndType & column = res.getByPosition(i + keys_size); ColumnWithTypeAndName & column = res.getByPosition(i + keys_size);
column.type = aggregate_functions[i]->getReturnType(); column.type = aggregate_functions[i]->getReturnType();
column.column = column.type->createColumn(); column.column = column.type->createColumn();
column.column->reserve(rows); column.column->reserve(rows);

View File

@ -17,6 +17,7 @@
#include <DB/Storages/MarkCache.h> #include <DB/Storages/MarkCache.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h> #include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
#include <DB/Storages/MergeTree/MergeList.h> #include <DB/Storages/MergeTree/MergeList.h>
#include <DB/Storages/MergeTree/MergeTreeSettings.h>
#include <DB/Storages/CompressionMethodSelector.h> #include <DB/Storages/CompressionMethodSelector.h>
#include <DB/Interpreters/Settings.h> #include <DB/Interpreters/Settings.h>
#include <DB/Interpreters/Users.h> #include <DB/Interpreters/Users.h>
@ -89,6 +90,7 @@ struct ContextShared
std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости. std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости.
std::unique_ptr<QueryLog> query_log; /// Для логгирования запросов. std::unique_ptr<QueryLog> query_log; /// Для логгирования запросов.
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector; /// Правила для выбора метода сжатия в зависимости от размера куска. mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector; /// Правила для выбора метода сжатия в зависимости от размера куска.
std::unique_ptr<MergeTreeSettings> merge_tree_settings; /// Настройки для движка MergeTree.
/// Кластеры для distributed таблиц /// Кластеры для distributed таблиц
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings /// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
@ -871,6 +873,21 @@ CompressionMethod Context::chooseCompressionMethod(size_t part_size, double part
} }
const MergeTreeSettings & Context::getMergeTreeSettings()
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (!shared->merge_tree_settings)
{
auto & config = Poco::Util::Application::instance().config();
shared->merge_tree_settings.reset(new MergeTreeSettings());
shared->merge_tree_settings->loadFromConfig("merge_tree", config);
}
return *shared->merge_tree_settings;
}
void Context::shutdown() void Context::shutdown()
{ {
shared->shutdown(); shared->shutdown();

View File

@ -3,13 +3,15 @@
#include <DB/Dictionaries/FlatDictionary.h> #include <DB/Dictionaries/FlatDictionary.h>
#include <DB/Dictionaries/HashedDictionary.h> #include <DB/Dictionaries/HashedDictionary.h>
#include <DB/Dictionaries/CacheDictionary.h> #include <DB/Dictionaries/CacheDictionary.h>
#include <DB/Dictionaries/RangeHashedDictionary.h>
#include <DB/Dictionaries/DictionaryStructure.h> #include <DB/Dictionaries/DictionaryStructure.h>
#include <memory> #include <memory>
#include <Yandex/singleton.h>
namespace DB namespace DB
{ {
DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::AbstractConfiguration & config, DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, Context & context) const const std::string & config_prefix, Context & context) const
{ {
@ -31,24 +33,44 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
const auto & layout_type = keys.front(); const auto & layout_type = keys.front();
if ("flat" == layout_type) if ("range_hashed" == layout_type)
{ {
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime); if (dict_struct.range_min.empty() || dict_struct.range_min.empty())
}
else if ("hashed" == layout_type)
{
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
}
else if ("cache" == layout_type)
{
const auto size = config.getInt(layout_prefix + ".cache.size_in_cells");
if (size == 0)
throw Exception{ throw Exception{
"Dictionary of type 'cache' cannot have 0 cells", "Dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max",
ErrorCodes::TOO_SMALL_BUFFER_SIZE ErrorCodes::BAD_ARGUMENTS
}; };
return std::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size); return std::make_unique<RangeHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
}
else
{
if (!dict_struct.range_min.empty() || !dict_struct.range_min.empty())
throw Exception{
"Elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS
};
if ("flat" == layout_type)
{
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
}
else if ("hashed" == layout_type)
{
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
}
else if ("cache" == layout_type)
{
const auto size = config.getInt(layout_prefix + ".cache.size_in_cells");
if (size == 0)
throw Exception{
"Dictionary of layout 'cache' cannot have 0 cells",
ErrorCodes::TOO_SMALL_BUFFER_SIZE
};
return std::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
}
} }
throw Exception{ throw Exception{
@ -57,4 +79,5 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
}; };
}; };
} }

View File

@ -61,7 +61,7 @@ ExpressionActions::Actions ExpressionAction::getPrerequisites(Block & sample_blo
if (sample_block.has(result_name)) if (sample_block.has(result_name))
throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN);
ColumnsWithNameAndType arguments(argument_names.size()); ColumnsWithTypeAndName arguments(argument_names.size());
for (size_t i = 0; i < argument_names.size(); ++i) for (size_t i = 0; i < argument_names.size(); ++i)
{ {
if (!sample_block.has(argument_names[i])) if (!sample_block.has(argument_names[i]))
@ -126,7 +126,7 @@ void ExpressionAction::prepare(Block & sample_block)
{ {
size_t result_position = sample_block.columns(); size_t result_position = sample_block.columns();
ColumnWithNameAndType new_column; ColumnWithTypeAndName new_column;
new_column.name = result_name; new_column.name = result_name;
new_column.type = result_type; new_column.type = result_type;
sample_block.insert(new_column); sample_block.insert(new_column);
@ -134,13 +134,13 @@ void ExpressionAction::prepare(Block & sample_block)
function->execute(sample_block, arguments, prerequisites, result_position); function->execute(sample_block, arguments, prerequisites, result_position);
/// Если получилась не константа, на всякий случай будем считать результат неизвестным. /// Если получилась не константа, на всякий случай будем считать результат неизвестным.
ColumnWithNameAndType & col = sample_block.getByPosition(result_position); ColumnWithTypeAndName & col = sample_block.getByPosition(result_position);
if (!col.column->isConst()) if (!col.column->isConst())
col.column = nullptr; col.column = nullptr;
} }
else else
{ {
sample_block.insert(ColumnWithNameAndType(nullptr, result_type, result_name)); sample_block.insert(ColumnWithTypeAndName(nullptr, result_type, result_name));
} }
break; break;
@ -150,7 +150,7 @@ void ExpressionAction::prepare(Block & sample_block)
{ {
for (NameSet::iterator it = array_joined_columns.begin(); it != array_joined_columns.end(); ++it) for (NameSet::iterator it = array_joined_columns.begin(); it != array_joined_columns.end(); ++it)
{ {
ColumnWithNameAndType & current = sample_block.getByName(*it); ColumnWithTypeAndName & current = sample_block.getByName(*it);
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*current.type); const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*current.type);
if (!array_type) if (!array_type)
throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH); throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH);
@ -164,7 +164,7 @@ void ExpressionAction::prepare(Block & sample_block)
case JOIN: case JOIN:
{ {
for (const auto & col : columns_added_by_join) for (const auto & col : columns_added_by_join)
sample_block.insert(ColumnWithNameAndType(nullptr, col.type, col.name)); sample_block.insert(ColumnWithTypeAndName(nullptr, col.type, col.name));
break; break;
} }
@ -177,7 +177,7 @@ void ExpressionAction::prepare(Block & sample_block)
{ {
const std::string & name = projection[i].first; const std::string & name = projection[i].first;
const std::string & alias = projection[i].second; const std::string & alias = projection[i].second;
ColumnWithNameAndType column = sample_block.getByName(name); ColumnWithTypeAndName column = sample_block.getByName(name);
if (alias != "") if (alias != "")
column.name = alias; column.name = alias;
new_block.insert(column); new_block.insert(column);
@ -198,14 +198,14 @@ void ExpressionAction::prepare(Block & sample_block)
if (sample_block.has(result_name)) if (sample_block.has(result_name))
throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN);
sample_block.insert(ColumnWithNameAndType(added_column, result_type, result_name)); sample_block.insert(ColumnWithTypeAndName(added_column, result_type, result_name));
break; break;
} }
case COPY_COLUMN: case COPY_COLUMN:
{ {
result_type = sample_block.getByName(source_name).type; result_type = sample_block.getByName(source_name).type;
sample_block.insert(ColumnWithNameAndType(sample_block.getByName(source_name).column, result_type, result_name)); sample_block.insert(ColumnWithTypeAndName(sample_block.getByName(source_name).column, result_type, result_name));
break; break;
} }
@ -246,7 +246,7 @@ void ExpressionAction::execute(Block & block) const
prerequisites[i] = block.getPositionByName(prerequisite_names[i]); prerequisites[i] = block.getPositionByName(prerequisite_names[i]);
} }
ColumnWithNameAndType new_column; ColumnWithTypeAndName new_column;
new_column.name = result_name; new_column.name = result_name;
new_column.type = result_type; new_column.type = result_type;
block.insert(new_column); block.insert(new_column);
@ -271,7 +271,7 @@ void ExpressionAction::execute(Block & block) const
size_t columns = block.columns(); size_t columns = block.columns();
for (size_t i = 0; i < columns; ++i) for (size_t i = 0; i < columns; ++i)
{ {
ColumnWithNameAndType & current = block.getByPosition(i); ColumnWithTypeAndName & current = block.getByPosition(i);
if (array_joined_columns.count(current.name)) if (array_joined_columns.count(current.name))
{ {
@ -312,7 +312,7 @@ void ExpressionAction::execute(Block & block) const
{ {
const std::string & name = projection[i].first; const std::string & name = projection[i].first;
const std::string & alias = projection[i].second; const std::string & alias = projection[i].second;
ColumnWithNameAndType column = block.getByName(name); ColumnWithTypeAndName column = block.getByName(name);
if (alias != "") if (alias != "")
column.name = alias; column.name = alias;
new_block.insert(column); new_block.insert(column);
@ -328,11 +328,11 @@ void ExpressionAction::execute(Block & block) const
break; break;
case ADD_COLUMN: case ADD_COLUMN:
block.insert(ColumnWithNameAndType(added_column->cloneResized(block.rowsInFirstColumn()), result_type, result_name)); block.insert(ColumnWithTypeAndName(added_column->cloneResized(block.rowsInFirstColumn()), result_type, result_name));
break; break;
case COPY_COLUMN: case COPY_COLUMN:
block.insert(ColumnWithNameAndType(block.getByName(source_name).column, result_type, result_name)); block.insert(ColumnWithTypeAndName(block.getByName(source_name).column, result_type, result_name));
break; break;
default: default:
@ -446,7 +446,7 @@ void ExpressionActions::checkLimits(Block & block) const
} }
} }
void ExpressionActions::addInput(const ColumnWithNameAndType & column) void ExpressionActions::addInput(const ColumnWithTypeAndName & column)
{ {
input_columns.emplace_back(column.name, column.type); input_columns.emplace_back(column.name, column.type);
sample_block.insert(column); sample_block.insert(column);
@ -454,7 +454,7 @@ void ExpressionActions::addInput(const ColumnWithNameAndType & column)
void ExpressionActions::addInput(const NameAndTypePair & column) void ExpressionActions::addInput(const NameAndTypePair & column)
{ {
addInput(ColumnWithNameAndType(nullptr, column.type, column.name)); addInput(ColumnWithTypeAndName(nullptr, column.type, column.name));
} }
void ExpressionActions::add(const ExpressionAction & action, Names & out_new_columns) void ExpressionActions::add(const ExpressionAction & action, Names & out_new_columns)
@ -573,7 +573,7 @@ void ExpressionActions::executeOnTotals(Block & block) const
{ {
for (const auto & name_and_type : input_columns) for (const auto & name_and_type : input_columns)
{ {
ColumnWithNameAndType elem(name_and_type.type->createColumn(), name_and_type.type, name_and_type.name); ColumnWithTypeAndName elem(name_and_type.type->createColumn(), name_and_type.type, name_and_type.name);
elem.column->insertDefault(); elem.column->insertDefault();
block.insert(elem); block.insert(elem);
} }
@ -903,7 +903,7 @@ BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRigh
{ {
Block left_sample_block; Block left_sample_block;
for (const auto & input_elem : input_columns) for (const auto & input_elem : input_columns)
left_sample_block.insert(ColumnWithNameAndType(nullptr, input_elem.type, input_elem.name)); left_sample_block.insert(ColumnWithTypeAndName(nullptr, input_elem.type, input_elem.name));
return action.join->createStreamWithNonJoinedRows(left_sample_block, max_block_size); return action.join->createStreamWithNonJoinedRows(left_sample_block, max_block_size);
} }
@ -918,7 +918,7 @@ void ExpressionActionsChain::addStep()
if (steps.empty()) if (steps.empty())
throw Exception("Cannot add action to empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR); throw Exception("Cannot add action to empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR);
ColumnsWithNameAndType columns = steps.back().actions->getSampleBlock().getColumns(); ColumnsWithTypeAndName columns = steps.back().actions->getSampleBlock().getColumns();
steps.push_back(Step(new ExpressionActions(columns, settings))); steps.push_back(Step(new ExpressionActions(columns, settings)));
} }

View File

@ -1011,7 +1011,7 @@ struct ExpressionAnalyzer::ScopeStack
stack.emplace_back(); stack.emplace_back();
Level & prev = stack[stack.size() - 2]; Level & prev = stack[stack.size() - 2];
ColumnsWithNameAndType all_columns; ColumnsWithTypeAndName all_columns;
NameSet new_names; NameSet new_names;
for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it) for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it)
@ -1024,7 +1024,7 @@ struct ExpressionAnalyzer::ScopeStack
const Block & prev_sample_block = prev.actions->getSampleBlock(); const Block & prev_sample_block = prev.actions->getSampleBlock();
for (size_t i = 0, size = prev_sample_block.columns(); i < size; ++i) for (size_t i = 0, size = prev_sample_block.columns(); i < size; ++i)
{ {
const ColumnWithNameAndType & col = prev_sample_block.unsafeGetByPosition(i); const ColumnWithTypeAndName & col = prev_sample_block.unsafeGetByPosition(i);
if (!new_names.count(col.name)) if (!new_names.count(col.name))
all_columns.push_back(col); all_columns.push_back(col);
} }
@ -1057,7 +1057,7 @@ struct ExpressionAnalyzer::ScopeStack
for (size_t i = 0; i < added.size(); ++i) for (size_t i = 0; i < added.size(); ++i)
{ {
const ColumnWithNameAndType & col = stack[level].actions->getSampleBlock().getByName(added[i]); const ColumnWithTypeAndName & col = stack[level].actions->getSampleBlock().getByName(added[i]);
for (size_t j = level + 1; j < stack.size(); ++j) for (size_t j = level + 1; j < stack.size(); ++j)
stack[j].actions->addInput(col); stack[j].actions->addInput(col);
} }
@ -1234,7 +1234,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
{ {
/// Мы в той части дерева, которую не собираемся вычислять. Нужно только определить типы. /// Мы в той части дерева, которую не собираемся вычислять. Нужно только определить типы.
/// Не будем выполнять подзапросы и составлять множества. Вставим произвольный столбец правильного типа. /// Не будем выполнять подзапросы и составлять множества. Вставим произвольный столбец правильного типа.
ColumnWithNameAndType fake_column; ColumnWithTypeAndName fake_column;
fake_column.name = node->getColumnName(); fake_column.name = node->getColumnName();
fake_column.type = new DataTypeUInt8; fake_column.type = new DataTypeUInt8;
actions_stack.addAction(ExpressionAction::addColumn(fake_column)); actions_stack.addAction(ExpressionAction::addColumn(fake_column));
@ -1275,7 +1275,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
} }
else if (set) else if (set)
{ {
ColumnWithNameAndType column; ColumnWithTypeAndName column;
column.type = new DataTypeSet; column.type = new DataTypeSet;
/// Если аргумент - множество, заданное перечислением значений, дадим ему уникальное имя, /// Если аргумент - множество, заданное перечислением значений, дадим ему уникальное имя,
@ -1370,7 +1370,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
/// потому что оно не однозначно определяет выражение (типы аргументов могут быть разными). /// потому что оно не однозначно определяет выражение (типы аргументов могут быть разными).
argument_names[i] = getUniqueName(actions_stack.getSampleBlock(), "__lambda"); argument_names[i] = getUniqueName(actions_stack.getSampleBlock(), "__lambda");
ColumnWithNameAndType lambda_column; ColumnWithTypeAndName lambda_column;
lambda_column.column = new ColumnExpression(1, lambda_actions, lambda_arguments, result_type, result_name); lambda_column.column = new ColumnExpression(1, lambda_actions, lambda_arguments, result_type, result_name);
lambda_column.type = argument_types[i]; lambda_column.type = argument_types[i];
lambda_column.name = argument_names[i]; lambda_column.name = argument_names[i];
@ -1400,7 +1400,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
{ {
DataTypePtr type = apply_visitor(FieldToDataType(), node->value); DataTypePtr type = apply_visitor(FieldToDataType(), node->value);
ColumnWithNameAndType column; ColumnWithTypeAndName column;
column.column = type->createConstColumn(1, node->value); column.column = type->createConstColumn(1, node->value);
column.type = type; column.type = type;
column.name = node->getColumnName(); column.name = node->getColumnName();
@ -1965,7 +1965,7 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd
else if (typeid_cast<const ASTSubquery *>(node.table.get())) else if (typeid_cast<const ASTSubquery *>(node.table.get()))
{ {
const auto & subquery = node.table->children.at(0); const auto & subquery = node.table->children.at(0);
nested_result_sample = InterpreterSelectQuery(subquery, context, QueryProcessingStage::Complete, subquery_depth + 1).getSampleBlock(); nested_result_sample = InterpreterSelectQuery::getSampleBlock(subquery, context);
} }
auto & keys = typeid_cast<ASTExpressionList &>(*node.using_expr_list); auto & keys = typeid_cast<ASTExpressionList &>(*node.using_expr_list);

Some files were not shown because too many files have changed in this diff Show More