mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge
This commit is contained in:
commit
d6c9ba676e
@ -15,6 +15,7 @@
|
||||
#include <DB/Interpreters/AggregationCommon.h>
|
||||
#include <DB/Common/HashTable/HashSet.h>
|
||||
#include <DB/Common/HyperLogLogWithSmallSetOptimization.h>
|
||||
#include <DB/Common/CombinedCardinalityEstimator.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
|
||||
{
|
||||
/** Структура для делегации работы по добавлению одного элемента в агрегатные функции uniq.
|
||||
@ -166,6 +187,28 @@ namespace detail
|
||||
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));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
|
@ -176,6 +176,7 @@ public:
|
||||
|
||||
/** Размножить все значения столько раз, сколько прописано в offsets.
|
||||
* (i-е значение размножается в offsets[i] - offsets[i - 1] значений.)
|
||||
* Необходимо для реализации операции ARRAY JOIN.
|
||||
*/
|
||||
typedef UInt64 Offset_t;
|
||||
typedef PODArray<Offset_t> Offsets_t;
|
||||
|
262
dbms/include/DB/Common/CombinedCardinalityEstimator.h
Normal file
262
dbms/include/DB/Common/CombinedCardinalityEstimator.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -47,7 +47,7 @@ public:
|
||||
|
||||
for (size_t i = 0; i < structure.size(); ++i)
|
||||
{
|
||||
ColumnWithNameAndType column;
|
||||
ColumnWithTypeAndName column;
|
||||
column.name = structure[i].first;
|
||||
column.type = data_type_factory.get(structure[i].second);
|
||||
column.column = column.type->createColumn();
|
||||
|
@ -1,51 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Common/HashTable/HashMap.h>
|
||||
|
||||
|
||||
/** Хеш-таблица, позволяющая очищать таблицу за O(1).
|
||||
* Еще более простая, чем HashMap: Key и Mapped должны быть POD-типами.
|
||||
*
|
||||
* Вместо этого класса можно было бы просто использовать в HashMap в качестве ключа пару <версия, ключ>,
|
||||
* но тогда таблица накапливала бы все ключи, которые в нее когда-либо складывали, и неоправданно росла.
|
||||
* Этот класс идет на шаг дальше и считает ключи со старой версией пустыми местами в хеш-таблице.
|
||||
*/
|
||||
|
||||
struct ClearableHashMapState
|
||||
{
|
||||
UInt32 version = 1;
|
||||
|
||||
/// Сериализация, в бинарном и текстовом виде.
|
||||
void write(DB::WriteBuffer & wb) const { DB::writeBinary(version, wb); }
|
||||
void writeText(DB::WriteBuffer & wb) const { DB::writeText(version, wb); }
|
||||
|
||||
/// Десериализация, в бинарном и текстовом виде.
|
||||
void read(DB::ReadBuffer & rb) { DB::readBinary(version, rb); }
|
||||
void readText(DB::ReadBuffer & rb) { DB::readText(version, rb); }
|
||||
};
|
||||
#include <DB/Common/HashTable/ClearableHashSet.h>
|
||||
|
||||
|
||||
template <typename Key, typename Mapped, typename Hash>
|
||||
struct ClearableHashMapCell : public HashMapCell<Key, Mapped, Hash, ClearableHashMapState>
|
||||
struct ClearableHashMapCell : public ClearableHashTableCell<Key, HashMapCell<Key, Mapped, Hash, ClearableHashSetState>>
|
||||
{
|
||||
typedef ClearableHashMapState State;
|
||||
typedef HashMapCell<Key, Mapped, Hash, ClearableHashMapState> Base;
|
||||
typedef typename Base::value_type value_type;
|
||||
using Base = ClearableHashTableCell<Key, HashMapCell<Key, Mapped, Hash, ClearableHashSetState>>;
|
||||
using Base::Base;
|
||||
|
||||
UInt32 version;
|
||||
|
||||
ClearableHashMapCell() {}
|
||||
ClearableHashMapCell(const Key & key_, const State & state) : Base(key_, state), version(state.version) {}
|
||||
ClearableHashMapCell(const value_type & value_, const State & state) : Base(value_, state), version(state.version) {}
|
||||
|
||||
bool isZero(const State & state) const { return version != state.version; }
|
||||
static bool isZero(const Key & key, const State & state) { return false; }
|
||||
|
||||
/// Установить значение ключа в ноль.
|
||||
void setZero() { version = 0; }
|
||||
|
||||
/// Нужно ли хранить нулевой ключ отдельно (то есть, могут ли в хэш-таблицу вставить нулевой ключ).
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
ClearableHashMapCell(const typename Base::value_type & value_, const typename Base::State & state)
|
||||
: Base::BaseCell(value_, state), Base::version(state.version) {}
|
||||
};
|
||||
|
||||
|
||||
|
70
dbms/include/DB/Common/HashTable/ClearableHashSet.h
Normal file
70
dbms/include/DB/Common/HashTable/ClearableHashSet.h
Normal file
@ -0,0 +1,70 @@
|
||||
#pragma once
|
||||
|
||||
#include <type_traits>
|
||||
#include <DB/Common/HashTable/HashSet.h>
|
||||
|
||||
|
||||
/** Хеш-таблица, позволяющая очищать таблицу за O(1).
|
||||
* Еще более простая, чем HashSet: Key и Mapped должны быть POD-типами.
|
||||
*
|
||||
* Вместо этого класса можно было бы просто использовать в HashSet в качестве ключа пару <версия, ключ>,
|
||||
* но тогда таблица накапливала бы все ключи, которые в нее когда-либо складывали, и неоправданно росла.
|
||||
* Этот класс идет на шаг дальше и считает ключи со старой версией пустыми местами в хеш-таблице.
|
||||
*/
|
||||
|
||||
|
||||
struct ClearableHashSetState
|
||||
{
|
||||
UInt32 version = 1;
|
||||
|
||||
/// Сериализация, в бинарном и текстовом виде.
|
||||
void write(DB::WriteBuffer & wb) const { DB::writeBinary(version, wb); }
|
||||
void writeText(DB::WriteBuffer & wb) const { DB::writeText(version, wb); }
|
||||
|
||||
/// Десериализация, в бинарном и текстовом виде.
|
||||
void read(DB::ReadBuffer & rb) { DB::readBinary(version, rb); }
|
||||
void readText(DB::ReadBuffer & rb) { DB::readText(version, rb); }
|
||||
};
|
||||
|
||||
|
||||
template <typename Key, typename BaseCell>
|
||||
struct ClearableHashTableCell : public BaseCell
|
||||
{
|
||||
typedef ClearableHashSetState State;
|
||||
typedef typename BaseCell::value_type value_type;
|
||||
|
||||
UInt32 version;
|
||||
|
||||
bool isZero(const State & state) const { return version != state.version; }
|
||||
static bool isZero(const Key & key, const State & state) { return false; }
|
||||
|
||||
/// Установить значение ключа в ноль.
|
||||
void setZero() { version = 0; }
|
||||
|
||||
/// Нужно ли хранить нулевой ключ отдельно (то есть, могут ли в хэш-таблицу вставить нулевой ключ).
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
|
||||
ClearableHashTableCell() {}
|
||||
ClearableHashTableCell(const Key & key_, const State & state) : BaseCell(key_, state), version(state.version) {}
|
||||
};
|
||||
|
||||
|
||||
template
|
||||
<
|
||||
typename Key,
|
||||
typename Hash = DefaultHash<Key>,
|
||||
typename Grower = HashTableGrower<>,
|
||||
typename Allocator = HashTableAllocator
|
||||
>
|
||||
class ClearableHashSet : public HashTable<Key, ClearableHashTableCell<Key, HashTableCell<Key, Hash, ClearableHashSetState>>, Hash, Grower, Allocator>
|
||||
{
|
||||
public:
|
||||
typedef Key key_type;
|
||||
typedef typename ClearableHashSet::cell_type::value_type value_type;
|
||||
|
||||
void clear()
|
||||
{
|
||||
++this->version;
|
||||
this->m_size = 0;
|
||||
}
|
||||
};
|
@ -102,9 +102,7 @@ struct HashMapCellWithSavedHash : public HashMapCell<Key, TMapped, Hash, TState>
|
||||
|
||||
size_t saved_hash;
|
||||
|
||||
HashMapCellWithSavedHash() : Base() {}
|
||||
HashMapCellWithSavedHash(const Key & key_, const typename Base::State & state) : Base(key_, state) {}
|
||||
HashMapCellWithSavedHash(const typename Base::value_type & value_, const typename Base::State & state) : Base(value_, state) {}
|
||||
using Base::Base;
|
||||
|
||||
bool keyEquals(const Key & key_) const { return this->value.first == key_; }
|
||||
bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->value.first == key_; }
|
||||
|
@ -757,7 +757,7 @@ public:
|
||||
{
|
||||
Cell x;
|
||||
x.read(rb);
|
||||
insert(x);
|
||||
insert(Cell::getKey(x.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
@ -781,7 +781,7 @@ public:
|
||||
Cell x;
|
||||
DB::assertString(",", rb);
|
||||
x.readText(rb);
|
||||
insert(x);
|
||||
insert(Cell::getKey(x.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -37,7 +37,7 @@ template<typename T1>
|
||||
std::multiset<T1> extractSingleValueFromBlock(const Block & block, const String & name)
|
||||
{
|
||||
std::multiset<T1> res;
|
||||
const ColumnWithNameAndType & data = block.getByName(name);
|
||||
const ColumnWithTypeAndName & data = block.getByName(name);
|
||||
size_t rows = block.rows();
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
res.insert((*data.column)[i].get<T1>());
|
||||
|
@ -6,11 +6,11 @@
|
||||
#include <initializer_list>
|
||||
|
||||
#include <DB/Core/BlockInfo.h>
|
||||
#include <DB/Core/ColumnWithNameAndType.h>
|
||||
#include <DB/Core/ColumnWithTypeAndName.h>
|
||||
#include <DB/Core/NamesAndTypes.h>
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
#include "ColumnsWithNameAndType.h"
|
||||
#include "ColumnsWithTypeAndName.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -26,7 +26,7 @@ class Context;
|
||||
class Block
|
||||
{
|
||||
public:
|
||||
typedef std::list<ColumnWithNameAndType> Container_t;
|
||||
typedef std::list<ColumnWithTypeAndName> Container_t;
|
||||
typedef std::vector<Container_t::iterator> IndexByPosition_t;
|
||||
typedef std::map<String, Container_t::iterator> IndexByName_t;
|
||||
|
||||
@ -39,7 +39,7 @@ public:
|
||||
BlockInfo info;
|
||||
|
||||
Block() = default;
|
||||
Block(std::initializer_list<ColumnWithNameAndType> il) : data{il}
|
||||
Block(std::initializer_list<ColumnWithTypeAndName> il) : data{il}
|
||||
{
|
||||
index_by_position.reserve(il.size());
|
||||
for (auto it = std::begin(data); it != std::end(data); ++it)
|
||||
@ -56,11 +56,11 @@ public:
|
||||
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);
|
||||
/// удалить столбец с заданным именем
|
||||
@ -68,20 +68,20 @@ public:
|
||||
/// Добавляет в блок недостающие столбцы со значениями по-умолчанию
|
||||
void addDefaults(const NamesAndTypesList & required_columns);
|
||||
|
||||
ColumnWithNameAndType & getByPosition(size_t position);
|
||||
const ColumnWithNameAndType & getByPosition(size_t position) const;
|
||||
ColumnWithTypeAndName & getByPosition(size_t position);
|
||||
const ColumnWithTypeAndName & getByPosition(size_t position) const;
|
||||
|
||||
ColumnWithNameAndType & unsafeGetByPosition(size_t position) { return *index_by_position[position]; }
|
||||
const ColumnWithNameAndType & unsafeGetByPosition(size_t position) const { return *index_by_position[position]; }
|
||||
ColumnWithTypeAndName & unsafeGetByPosition(size_t position) { return *index_by_position[position]; }
|
||||
const ColumnWithTypeAndName & unsafeGetByPosition(size_t position) const { return *index_by_position[position]; }
|
||||
|
||||
ColumnWithNameAndType & getByName(const std::string & name);
|
||||
const ColumnWithNameAndType & getByName(const std::string & name) const;
|
||||
ColumnWithTypeAndName & getByName(const std::string & name);
|
||||
const ColumnWithTypeAndName & getByName(const std::string & name) const;
|
||||
|
||||
bool has(const std::string & name) const;
|
||||
|
||||
size_t getPositionByName(const std::string & name) const;
|
||||
|
||||
ColumnsWithNameAndType getColumns() const;
|
||||
ColumnsWithTypeAndName getColumns() const;
|
||||
NamesAndTypesList getColumnsList() const;
|
||||
|
||||
/** Возвращает количество строк в блоке.
|
||||
|
@ -14,19 +14,19 @@ using Poco::SharedPtr;
|
||||
/** Тип данных для представления столбца вместе с его типом и именем в оперативке.
|
||||
*/
|
||||
|
||||
struct ColumnWithNameAndType
|
||||
struct ColumnWithTypeAndName
|
||||
{
|
||||
ColumnPtr column;
|
||||
DataTypePtr type;
|
||||
String name;
|
||||
|
||||
ColumnWithNameAndType() {}
|
||||
ColumnWithNameAndType(const ColumnPtr & column_, const DataTypePtr & type_, const String name_)
|
||||
|
||||
ColumnWithTypeAndName() {}
|
||||
ColumnWithTypeAndName(const ColumnPtr & column_, const DataTypePtr & type_, const String name_)
|
||||
: column(column_), type(type_), name(name_) {}
|
||||
|
||||
ColumnWithNameAndType cloneEmpty() const
|
||||
ColumnWithTypeAndName cloneEmpty() const
|
||||
{
|
||||
ColumnWithNameAndType res;
|
||||
ColumnWithTypeAndName res;
|
||||
|
||||
res.name = name;
|
||||
res.type = type->clone();
|
@ -1,13 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
|
||||
#include <DB/Core/ColumnWithNameAndType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
typedef std::vector<ColumnWithNameAndType> ColumnsWithNameAndType;
|
||||
|
||||
}
|
13
dbms/include/DB/Core/ColumnsWithTypeAndName.h
Normal file
13
dbms/include/DB/Core/ColumnsWithTypeAndName.h
Normal file
@ -0,0 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
|
||||
#include <DB/Core/ColumnWithTypeAndName.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
typedef std::vector<ColumnWithTypeAndName> ColumnsWithTypeAndName;
|
||||
|
||||
}
|
@ -180,7 +180,6 @@ namespace ErrorCodes
|
||||
CANNOT_CREATE_DIRECTORY = 172,
|
||||
CANNOT_ALLOCATE_MEMORY = 173,
|
||||
CYCLIC_ALIASES = 174,
|
||||
NEGATIVE_REFCOUNT = 175,
|
||||
CHUNK_NOT_FOUND = 176,
|
||||
DUPLICATE_CHUNK_NAME = 177,
|
||||
MULTIPLE_ALIASES_FOR_EXPRESSION = 178,
|
||||
|
@ -43,14 +43,31 @@ inline bool compareSSE2(const char * p1, const char * 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)
|
||||
{
|
||||
while (size >= 64)
|
||||
{
|
||||
if ( compareSSE2(p1, p2)
|
||||
&& compareSSE2(p1 + 16, p2 + 16)
|
||||
&& compareSSE2(p1 + 32, p2 + 32)
|
||||
&& compareSSE2(p1 + 48, p2 + 48))
|
||||
if (compareSSE2x4(p1, p2))
|
||||
{
|
||||
p1 += 64;
|
||||
p2 += 64;
|
||||
|
@ -4,7 +4,7 @@
|
||||
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Core/ColumnWithNameAndType.h>
|
||||
#include <DB/Core/ColumnWithTypeAndName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -41,7 +41,7 @@ protected:
|
||||
if (!res)
|
||||
return res;
|
||||
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);
|
||||
return res;
|
||||
}
|
||||
|
@ -25,14 +25,14 @@ struct BlockIO
|
||||
Block out_sample; /// Пример блока, которого нужно писать в out.
|
||||
|
||||
/// Здесь могут быть установлены колбэки для логгирования запроса.
|
||||
std::function<void(IBlockInputStream &)> finish_callback;
|
||||
std::function<void(IBlockInputStream *)> finish_callback;
|
||||
std::function<void()> exception_callback;
|
||||
|
||||
/// Вызывайте эти функции, если нужно логгировать запрос.
|
||||
void onFinish()
|
||||
{
|
||||
if (in && finish_callback)
|
||||
finish_callback(*in);
|
||||
if (finish_callback)
|
||||
finish_callback(in.get());
|
||||
}
|
||||
|
||||
void onException()
|
||||
|
@ -14,10 +14,10 @@ class FormatFactory
|
||||
{
|
||||
public:
|
||||
BlockInputStreamPtr getInput(const String & name, ReadBuffer & buf,
|
||||
Block & sample, size_t max_block_size) const;
|
||||
const Block & sample, size_t max_block_size) const;
|
||||
|
||||
BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf,
|
||||
Block & sample) const;
|
||||
const Block & sample) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -21,17 +21,21 @@ class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream
|
||||
public:
|
||||
/** Столбцы из key_names и аргументы агрегатных функций, уже должны быть вычислены.
|
||||
*/
|
||||
ParallelAggregatingBlockInputStream(BlockInputStreams inputs, const Names & key_names,
|
||||
const AggregateDescriptions & aggregates, bool overflow_row_, bool final_, size_t max_threads_,
|
||||
ParallelAggregatingBlockInputStream(
|
||||
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_,
|
||||
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_,
|
||||
compiler_, min_count_to_compile_, group_by_two_level_threshold_),
|
||||
final(final_), max_threads(std::min(inputs.size(), max_threads_)),
|
||||
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"; }
|
||||
|
@ -45,8 +45,16 @@ template <typename Handler>
|
||||
class ParallelInputsProcessor
|
||||
{
|
||||
public:
|
||||
ParallelInputsProcessor(BlockInputStreams inputs_, size_t max_threads_, Handler & handler_)
|
||||
: inputs(inputs_), max_threads(std::min(inputs_.size(), max_threads_)), handler(handler_)
|
||||
/** additional_input_at_end - если не nullptr,
|
||||
* то из этого источника начинают доставаться блоки лишь после того, как все остальные источники обработаны.
|
||||
* Это делается в основном потоке.
|
||||
*
|
||||
* Предназначено для реализации 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)
|
||||
available_inputs.emplace(inputs_[i], i);
|
||||
@ -150,6 +158,25 @@ private:
|
||||
/// Последний поток при выходе сообщает, что данных больше нет.
|
||||
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();
|
||||
}
|
||||
}
|
||||
@ -206,6 +233,7 @@ private:
|
||||
}
|
||||
|
||||
BlockInputStreams inputs;
|
||||
BlockInputStreamPtr additional_input_at_end;
|
||||
unsigned max_threads;
|
||||
|
||||
Handler & handler;
|
||||
|
@ -84,6 +84,16 @@ public:
|
||||
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
|
||||
return;
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(external_tables_mutex);
|
||||
|
||||
/// Останавливаем отправку внешних данных.
|
||||
for (auto & vec : external_tables_data)
|
||||
for (auto & elem : vec)
|
||||
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(elem.first.get()))
|
||||
stream->cancel();
|
||||
}
|
||||
|
||||
if (!isQueryPending() || hasThrownException())
|
||||
return;
|
||||
|
||||
@ -107,27 +117,30 @@ protected:
|
||||
{
|
||||
size_t count = parallel_replicas->size();
|
||||
|
||||
std::vector<ExternalTablesData> instances;
|
||||
instances.reserve(count);
|
||||
|
||||
for (size_t i = 0; i < count; ++i)
|
||||
{
|
||||
ExternalTablesData res;
|
||||
for (const auto & table : external_tables)
|
||||
std::lock_guard<std::mutex> lock(external_tables_mutex);
|
||||
|
||||
external_tables_data.reserve(count);
|
||||
|
||||
for (size_t i = 0; i < count; ++i)
|
||||
{
|
||||
StoragePtr cur = table.second;
|
||||
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
|
||||
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context, settings,
|
||||
stage, DEFAULT_BLOCK_SIZE, 1);
|
||||
if (input.size() == 0)
|
||||
res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), table.first));
|
||||
else
|
||||
res.push_back(std::make_pair(input[0], table.first));
|
||||
ExternalTablesData res;
|
||||
for (const auto & table : external_tables)
|
||||
{
|
||||
StoragePtr cur = table.second;
|
||||
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
|
||||
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context, settings,
|
||||
stage, DEFAULT_BLOCK_SIZE, 1);
|
||||
if (input.size() == 0)
|
||||
res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), table.first));
|
||||
else
|
||||
res.push_back(std::make_pair(input[0], table.first));
|
||||
}
|
||||
external_tables_data.push_back(std::move(res));
|
||||
}
|
||||
instances.push_back(std::move(res));
|
||||
}
|
||||
|
||||
parallel_replicas->sendExternalTablesData(instances);
|
||||
parallel_replicas->sendExternalTablesData(external_tables_data);
|
||||
}
|
||||
|
||||
Block readImpl() override
|
||||
@ -302,6 +315,10 @@ private:
|
||||
QueryProcessingStage::Enum stage;
|
||||
Context context;
|
||||
|
||||
/// Потоки для чтения из временных таблиц - для последующей отправки данных на удалённые серверы для GLOBAL-подзапросов.
|
||||
std::vector<ExternalTablesData> external_tables_data;
|
||||
std::mutex external_tables_mutex;
|
||||
|
||||
/// Установили соединения с репликами, но ещё не отправили запрос.
|
||||
std::atomic<bool> established { false };
|
||||
|
||||
|
24
dbms/include/DB/DataStreams/TSKVRowOutputStream.h
Normal file
24
dbms/include/DB/DataStreams/TSKVRowOutputStream.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -28,12 +28,14 @@ using Poco::SharedPtr;
|
||||
class UnionBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
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)),
|
||||
handler(*this),
|
||||
processor(inputs, max_threads, handler)
|
||||
processor(inputs, additional_input_at_end, max_threads, handler)
|
||||
{
|
||||
children = inputs;
|
||||
if (additional_input_at_end)
|
||||
children.push_back(additional_input_at_end);
|
||||
}
|
||||
|
||||
String getName() const override { return "Union"; }
|
||||
|
@ -136,7 +136,7 @@ public:
|
||||
Block tuple_block;
|
||||
for (size_t i = 0, size = elems.size(); i < size; ++i)
|
||||
{
|
||||
ColumnWithNameAndType col;
|
||||
ColumnWithTypeAndName col;
|
||||
col.column = elems[i]->createColumn();
|
||||
col.type = elems[i]->clone();
|
||||
tuple_block.insert(col);
|
||||
|
@ -27,7 +27,7 @@ public:
|
||||
|
||||
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;
|
||||
|
@ -87,61 +87,11 @@ public:
|
||||
|
||||
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
|
||||
{
|
||||
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)\
|
||||
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
|
||||
{\
|
||||
|
@ -95,6 +95,14 @@ private:
|
||||
|
||||
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)
|
||||
{
|
||||
writeString(", ", out);
|
||||
|
@ -18,7 +18,7 @@ namespace
|
||||
Block createSampleBlock(const DictionaryStructure & dict_struct)
|
||||
{
|
||||
Block block{
|
||||
ColumnWithNameAndType{
|
||||
ColumnWithTypeAndName{
|
||||
new ColumnUInt64,
|
||||
new DataTypeUInt64,
|
||||
dict_struct.id_name
|
||||
@ -26,7 +26,7 @@ Block createSampleBlock(const DictionaryStructure & dict_struct)
|
||||
};
|
||||
|
||||
for (const auto & attribute : dict_struct.attributes)
|
||||
block.insert(ColumnWithNameAndType{
|
||||
block.insert(ColumnWithTypeAndName{
|
||||
attribute.type->createColumn(), attribute.type, attribute.name
|
||||
});
|
||||
|
||||
|
@ -117,9 +117,13 @@ struct DictionaryStructure final
|
||||
{
|
||||
std::string id_name;
|
||||
std::vector<DictionaryAttribute> attributes;
|
||||
std::string range_min;
|
||||
std::string range_max;
|
||||
|
||||
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())
|
||||
throw Exception{
|
||||
|
@ -81,64 +81,11 @@ public:
|
||||
|
||||
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
|
||||
{
|
||||
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)\
|
||||
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
|
||||
{\
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <memory>
|
||||
#include <tuple>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -78,67 +79,11 @@ public:
|
||||
|
||||
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
|
||||
{
|
||||
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)\
|
||||
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
|
||||
{\
|
||||
|
@ -12,16 +12,15 @@ namespace DB
|
||||
|
||||
class IDictionarySource;
|
||||
|
||||
class IDictionary;
|
||||
using DictionaryPtr = std::unique_ptr<IDictionary>;
|
||||
class IDictionaryBase;
|
||||
using DictionaryPtr = std::unique_ptr<IDictionaryBase>;
|
||||
|
||||
class DictionaryLifetime;
|
||||
class DictionaryStructure;
|
||||
class ColumnString;
|
||||
|
||||
class IDictionary
|
||||
struct IDictionaryBase
|
||||
{
|
||||
public:
|
||||
using id_t = std::uint64_t;
|
||||
|
||||
virtual std::exception_ptr getCreationException() const = 0;
|
||||
@ -53,10 +52,24 @@ public:
|
||||
|
||||
virtual bool isInjective(const std::string & attribute_name) const = 0;
|
||||
|
||||
virtual ~IDictionaryBase() = default;
|
||||
};
|
||||
|
||||
struct IDictionary : IDictionaryBase
|
||||
{
|
||||
virtual bool hasHierarchy() const = 0;
|
||||
|
||||
/// 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;
|
||||
|
||||
bool in(id_t child_id, const id_t ancestor_id) const
|
||||
@ -67,20 +80,7 @@ public:
|
||||
return child_id != 0;
|
||||
}
|
||||
|
||||
/// functions for individual access
|
||||
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
|
||||
/// return mapped values for a collection of identifiers
|
||||
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 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 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 ~IDictionary() = default;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -106,6 +106,14 @@ private:
|
||||
|
||||
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)
|
||||
{
|
||||
writeString(", ", out);
|
||||
|
427
dbms/include/DB/Dictionaries/RangeHashedDictionary.h
Normal file
427
dbms/include/DB/Dictionaries/RangeHashedDictionary.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -33,14 +33,21 @@ namespace DB
|
||||
* has(arr, x) - есть ли в массиве элемент x.
|
||||
* indexOf(arr, x) - возвращает индекс элемента x (начиная с 1), если он есть в массиве, или 0, если его нет.
|
||||
* arrayEnumerate(arr) - возаращает массив [1,2,3,..., length(arr)]
|
||||
* arrayEnumerateUniq(arr) - возаращает массив, параллельный данному, где для каждого элемента указано,
|
||||
* какой он по счету среди элементов с таким значением.
|
||||
* Например: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]
|
||||
*
|
||||
* arrayUniq(arr) - считает количество разных элементов в массиве,
|
||||
* arrayUniq(arr1, arr2, ...) - считает количество разных кортежей из элементов на соответствующих позициях в нескольких массивах.
|
||||
*
|
||||
* arrayEnumerateUniq(arr)
|
||||
* - возаращает массив, параллельный данному, где для каждого элемента указано,
|
||||
* какой он по счету среди элементов с таким значением.
|
||||
* Например: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]
|
||||
* arrayEnumerateUniq(arr1, arr2...)
|
||||
* - для кортежей из элементов на соответствующих позициях в нескольких массивах.
|
||||
*
|
||||
* emptyArrayToSingle(arr) - заменить пустые массивы на массивы из одного элемента со значением "по-умолчанию".
|
||||
*/
|
||||
|
||||
|
||||
|
||||
|
||||
class FunctionArray : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -634,12 +641,12 @@ private:
|
||||
|
||||
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.type = new DataTypeArray(tuple_block.getByPosition(i).type);
|
||||
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);
|
||||
|
||||
execute(block_of_temporary_results, ColumnNumbers{i * 2 + 1, 0}, i * 2 + 2);
|
||||
@ -995,11 +1002,11 @@ public:
|
||||
{
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
|
||||
ColumnVector<UInt32> * res_nested = new ColumnVector<UInt32>;
|
||||
ColumnUInt32 * res_nested = new ColumnUInt32;
|
||||
ColumnArray * res_array = new ColumnArray(res_nested, array->getOffsetsColumn());
|
||||
block.getByPosition(result).column = res_array;
|
||||
|
||||
ColumnVector<UInt32>::Container_t & res_values = res_nested->getData();
|
||||
ColumnUInt32::Container_t & res_values = res_nested->getData();
|
||||
res_values.resize(array->getData().size());
|
||||
size_t prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
@ -1034,6 +1041,239 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Считает количество разных элементов в массиве, или количество разных кортежей из элементов на соответствующих позициях в нескольких массивах.
|
||||
/// NOTE Реализация частично совпадает с arrayEnumerateUniq.
|
||||
class FunctionArrayUniq : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayUniq";
|
||||
static IFunction * create(const Context & context) { return new FunctionArrayUniq; }
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() == 0)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be at least 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*arguments[i]);
|
||||
if (!array_type)
|
||||
throw Exception("All arguments for function " + getName() + " must be arrays; argument " + toString(i + 1) + " isn't.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return new DataTypeUInt32;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (arguments.size() == 1 && executeConst(block, arguments, result))
|
||||
return;
|
||||
|
||||
Columns array_columns(arguments.size());
|
||||
const ColumnArray::Offsets_t * offsets = nullptr;
|
||||
ConstColumnPlainPtrs data_columns(arguments.size());
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
ColumnPtr array_ptr = block.getByPosition(arguments[i]).column;
|
||||
const ColumnArray * array = typeid_cast<const ColumnArray *>(&*array_ptr);
|
||||
if (!array)
|
||||
{
|
||||
const ColumnConstArray * const_array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[i]).column);
|
||||
if (!const_array)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName()
|
||||
+ " of " + toString(i + 1) + "-th argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
array_ptr = const_array->convertToFullColumn();
|
||||
array = typeid_cast<const ColumnArray *>(&*array_ptr);
|
||||
}
|
||||
array_columns[i] = array_ptr;
|
||||
const ColumnArray::Offsets_t & offsets_i = array->getOffsets();
|
||||
if (!i)
|
||||
offsets = &offsets_i;
|
||||
else if (offsets_i != *offsets)
|
||||
throw Exception("Lengths of all arrays passsed to " + getName() + " must be equal.",
|
||||
ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
data_columns[i] = &array->getData();
|
||||
}
|
||||
|
||||
const ColumnArray * first_array = typeid_cast<const ColumnArray *>(&*array_columns[0]);
|
||||
ColumnUInt32 * res = new ColumnUInt32;
|
||||
block.getByPosition(result).column = res;
|
||||
|
||||
ColumnUInt32::Container_t & res_values = res->getData();
|
||||
res_values.resize(offsets->size());
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (!( executeNumber<UInt8> (first_array, res_values)
|
||||
|| executeNumber<UInt16> (first_array, res_values)
|
||||
|| executeNumber<UInt32> (first_array, res_values)
|
||||
|| executeNumber<UInt64> (first_array, res_values)
|
||||
|| executeNumber<Int8> (first_array, res_values)
|
||||
|| executeNumber<Int16> (first_array, res_values)
|
||||
|| executeNumber<Int32> (first_array, res_values)
|
||||
|| executeNumber<Int64> (first_array, res_values)
|
||||
|| executeNumber<Float32> (first_array, res_values)
|
||||
|| executeNumber<Float64> (first_array, res_values)
|
||||
|| executeString (first_array, res_values)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!execute128bit(*offsets, data_columns, res_values))
|
||||
executeHashed(*offsets, data_columns, res_values);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
/// Изначально выделить кусок памяти для 512 элементов.
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(const ColumnArray * array, ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
const ColumnVector<T> * nested = typeid_cast<const ColumnVector<T> *>(&array->getData());
|
||||
if (!nested)
|
||||
return false;
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
const typename ColumnVector<T>::Container_t & values = nested->getData();
|
||||
|
||||
typedef ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)> > Set;
|
||||
|
||||
Set set;
|
||||
size_t prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
set.insert(values[j]);
|
||||
|
||||
res_values[i] = set.size();
|
||||
prev_off = off;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeString(const ColumnArray * array, ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
const ColumnString * nested = typeid_cast<const ColumnString *>(&array->getData());
|
||||
if (!nested)
|
||||
return false;
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
|
||||
typedef ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)> > Set;
|
||||
|
||||
Set set;
|
||||
size_t prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
set.insert(nested->getDataAt(j));
|
||||
|
||||
res_values[i] = set.size();
|
||||
prev_off = off;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnConstArray * array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
if (!array)
|
||||
return false;
|
||||
const Array & values = array->getData();
|
||||
|
||||
std::set<Field> set;
|
||||
for (size_t i = 0; i < values.size(); ++i)
|
||||
set.insert(values[i]);
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstUInt32(array->size(), set.size());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool execute128bit(
|
||||
const ColumnArray::Offsets_t & offsets,
|
||||
const ConstColumnPlainPtrs & columns,
|
||||
ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
size_t keys_bytes = 0;
|
||||
Sizes key_sizes(count);
|
||||
for (size_t j = 0; j < count; ++j)
|
||||
{
|
||||
if (!columns[j]->isFixed())
|
||||
return false;
|
||||
key_sizes[j] = columns[j]->sizeOfField();
|
||||
keys_bytes += key_sizes[j];
|
||||
}
|
||||
if (keys_bytes > 16)
|
||||
return false;
|
||||
|
||||
typedef ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > Set;
|
||||
|
||||
Set set;
|
||||
size_t prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
set.insert(packFixed<UInt128>(j, count, columns, key_sizes));
|
||||
|
||||
res_values[i] = set.size();
|
||||
prev_off = off;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void executeHashed(
|
||||
const ColumnArray::Offsets_t & offsets,
|
||||
const ConstColumnPlainPtrs & columns,
|
||||
ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
|
||||
typedef ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > Set;
|
||||
|
||||
Set set;
|
||||
size_t prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
set.insert(hash128(j, count, columns));
|
||||
|
||||
res_values[i] = set.size();
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionArrayEnumerateUniq : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -1100,11 +1340,11 @@ public:
|
||||
}
|
||||
|
||||
const ColumnArray * first_array = typeid_cast<const ColumnArray *>(&*array_columns[0]);
|
||||
ColumnVector<UInt32> * res_nested = new ColumnVector<UInt32>;
|
||||
ColumnUInt32 * res_nested = new ColumnUInt32;
|
||||
ColumnArray * res_array = new ColumnArray(res_nested, first_array->getOffsetsColumn());
|
||||
block.getByPosition(result).column = res_array;
|
||||
|
||||
ColumnVector<UInt32>::Container_t & res_values = res_nested->getData();
|
||||
ColumnUInt32::Container_t & res_values = res_nested->getData();
|
||||
if (!offsets->empty())
|
||||
res_values.resize(offsets->back());
|
||||
|
||||
@ -1137,7 +1377,7 @@ private:
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(const ColumnArray * array, ColumnVector<UInt32>::Container_t & res_values)
|
||||
bool executeNumber(const ColumnArray * array, ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
const ColumnVector<T> * nested = typeid_cast<const ColumnVector<T> *>(&array->getData());
|
||||
if (!nested)
|
||||
@ -1163,7 +1403,7 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeString(const ColumnArray * array, ColumnVector<UInt32>::Container_t & res_values)
|
||||
bool executeString(const ColumnArray * array, ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
const ColumnString * nested = typeid_cast<const ColumnString *>(&array->getData());
|
||||
if (!nested)
|
||||
@ -1211,7 +1451,7 @@ private:
|
||||
bool execute128bit(
|
||||
const ColumnArray::Offsets_t & offsets,
|
||||
const ConstColumnPlainPtrs & columns,
|
||||
ColumnVector<UInt32>::Container_t & res_values)
|
||||
ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
size_t keys_bytes = 0;
|
||||
@ -1248,7 +1488,7 @@ private:
|
||||
void executeHashed(
|
||||
const ColumnArray::Offsets_t & offsets,
|
||||
const ConstColumnPlainPtrs & columns,
|
||||
ColumnVector<UInt32>::Container_t & res_values)
|
||||
ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
|
||||
@ -1360,7 +1600,17 @@ private:
|
||||
{
|
||||
const auto & in_data = in->getData();
|
||||
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)
|
||||
throw Exception{
|
||||
"A call to function " + getName() + " would produce " + std::to_string(total_values) +
|
||||
@ -1391,6 +1641,12 @@ private:
|
||||
else if (const auto in = typeid_cast<const ColumnConst<T> *>(arg))
|
||||
{
|
||||
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;
|
||||
if (total_values > max_elements)
|
||||
throw Exception{
|
||||
@ -1441,13 +1697,263 @@ private:
|
||||
};
|
||||
|
||||
|
||||
class FunctionEmptyArrayToSingle : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "emptyArrayToSingle";
|
||||
static IFunction * create(const Context & context) { return new FunctionEmptyArrayToSingle; }
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("Argument for function " + getName() + " must be array.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0]->clone();
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (executeConst(block, arguments, result))
|
||||
return;
|
||||
|
||||
const ColumnArray * array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
if (!array)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
ColumnPtr res_ptr = array->cloneEmpty();
|
||||
block.getByPosition(result).column = res_ptr;
|
||||
ColumnArray & res = static_cast<ColumnArray &>(*res_ptr);
|
||||
|
||||
const IColumn & src_data = array->getData();
|
||||
const ColumnArray::Offsets_t & src_offsets = array->getOffsets();
|
||||
IColumn & res_data = res.getData();
|
||||
ColumnArray::Offsets_t & res_offsets = res.getOffsets();
|
||||
|
||||
if (!( executeNumber<UInt8> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeNumber<UInt16> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeNumber<UInt32> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeNumber<UInt64> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeNumber<Int8> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeNumber<Int16> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeNumber<Int32> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeNumber<Int64> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeNumber<Float32> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeNumber<Float64> (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeString (src_data, src_offsets, res_data, res_offsets)
|
||||
|| executeFixedString (src_data, src_offsets, res_data, res_offsets)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (const ColumnConstArray * const_array = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
if (const_array->getData().empty())
|
||||
{
|
||||
auto nested_type = typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType();
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstArray(
|
||||
block.rowsInFirstColumn(),
|
||||
{nested_type->getDefault()},
|
||||
nested_type->clone());
|
||||
}
|
||||
else
|
||||
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(
|
||||
const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets,
|
||||
IColumn & res_data_col, ColumnArray::Offsets_t & res_offsets)
|
||||
{
|
||||
if (const ColumnVector<T> * src_data_concrete = typeid_cast<const ColumnVector<T> *>(&src_data))
|
||||
{
|
||||
const PODArray<T> & src_data = src_data_concrete->getData();
|
||||
PODArray<T> & res_data = typeid_cast<ColumnVector<T> &>(res_data_col).getData();
|
||||
size_t size = src_offsets.size();
|
||||
res_offsets.resize(size);
|
||||
res_data.reserve(src_data.size());
|
||||
|
||||
ColumnArray::Offset_t src_prev_offset = 0;
|
||||
ColumnArray::Offset_t res_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (src_offsets[i] != src_prev_offset)
|
||||
{
|
||||
size_t size_to_write = src_offsets[i] - src_prev_offset;
|
||||
size_t prev_res_data_size = res_data.size();
|
||||
res_data.resize(prev_res_data_size + size_to_write);
|
||||
memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset], size_to_write * sizeof(T));
|
||||
res_prev_offset += size_to_write;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
res_data.push_back(T());
|
||||
++res_prev_offset;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
bool executeFixedString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets,
|
||||
IColumn & res_data_col, ColumnArray::Offsets_t & res_offsets)
|
||||
{
|
||||
if (const ColumnFixedString * src_data_concrete = typeid_cast<const ColumnFixedString *>(&src_data))
|
||||
{
|
||||
const size_t n = src_data_concrete->getN();
|
||||
const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars();
|
||||
ColumnFixedString::Chars_t & res_data = typeid_cast<ColumnFixedString &>(res_data_col).getChars();
|
||||
size_t size = src_offsets.size();
|
||||
res_offsets.resize(size);
|
||||
res_data.reserve(src_data.size());
|
||||
|
||||
ColumnArray::Offset_t src_prev_offset = 0;
|
||||
ColumnArray::Offset_t res_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (src_offsets[i] != src_prev_offset)
|
||||
{
|
||||
size_t size_to_write = src_offsets[i] - src_prev_offset;
|
||||
size_t prev_res_data_size = res_data.size();
|
||||
res_data.resize(prev_res_data_size + size_to_write * n);
|
||||
memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset], size_to_write * n);
|
||||
res_prev_offset += size_to_write;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t prev_res_data_size = res_data.size();
|
||||
res_data.resize(prev_res_data_size + n);
|
||||
memset(&res_data[prev_res_data_size], 0, n);
|
||||
++res_prev_offset;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
bool executeString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets_t & src_array_offsets,
|
||||
IColumn & res_data_col, ColumnArray::Offsets_t & res_array_offsets)
|
||||
{
|
||||
if (const ColumnString * src_data_concrete = typeid_cast<const ColumnString *>(&src_data))
|
||||
{
|
||||
const ColumnString::Offsets_t & src_string_offsets = src_data_concrete->getOffsets();
|
||||
ColumnString::Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res_data_col).getOffsets();
|
||||
|
||||
const ColumnString::Chars_t & src_data = src_data_concrete->getChars();
|
||||
ColumnString::Chars_t & res_data = typeid_cast<ColumnString &>(res_data_col).getChars();
|
||||
|
||||
size_t size = src_array_offsets.size();
|
||||
res_array_offsets.resize(size);
|
||||
res_string_offsets.reserve(src_string_offsets.size());
|
||||
res_data.reserve(src_data.size());
|
||||
|
||||
ColumnArray::Offset_t src_array_prev_offset = 0;
|
||||
ColumnArray::Offset_t res_array_prev_offset = 0;
|
||||
|
||||
ColumnString::Offset_t src_string_prev_offset = 0;
|
||||
ColumnString::Offset_t res_string_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (src_array_offsets[i] != src_array_prev_offset)
|
||||
{
|
||||
size_t array_size = src_array_offsets[i] - src_array_prev_offset;
|
||||
|
||||
size_t bytes_to_copy = 0;
|
||||
size_t from_string_prev_offset_local = src_string_prev_offset;
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
size_t string_size = src_string_offsets[src_array_prev_offset + j] - from_string_prev_offset_local;
|
||||
|
||||
res_string_prev_offset += string_size;
|
||||
res_string_offsets.push_back(res_string_prev_offset);
|
||||
|
||||
from_string_prev_offset_local += string_size;
|
||||
bytes_to_copy += string_size;
|
||||
}
|
||||
|
||||
size_t res_data_old_size = res_data.size();
|
||||
res_data.resize(res_data_old_size + bytes_to_copy);
|
||||
memcpy(&res_data[res_data_old_size], &src_data[src_string_prev_offset], bytes_to_copy);
|
||||
|
||||
res_array_prev_offset += array_size;
|
||||
res_array_offsets[i] = res_array_prev_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
res_data.push_back(0); /// Пустая строка, включая ноль на конце.
|
||||
|
||||
++res_string_prev_offset;
|
||||
res_string_offsets.push_back(res_string_prev_offset);
|
||||
|
||||
++res_array_prev_offset;
|
||||
res_array_offsets[i] = res_array_prev_offset;
|
||||
}
|
||||
|
||||
src_array_prev_offset = src_array_offsets[i];
|
||||
|
||||
if (src_array_prev_offset)
|
||||
src_string_prev_offset = src_string_offsets[src_array_prev_offset - 1];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NameHas { static constexpr auto name = "has"; };
|
||||
struct NameIndexOf { static constexpr auto name = "indexOf"; };
|
||||
struct NameCountEqual { static constexpr auto name = "countEqual"; };
|
||||
|
||||
typedef FunctionArrayIndex<IndexToOne, NameHas> FunctionHas;
|
||||
typedef FunctionArrayIndex<IndexToOne, NameHas> FunctionHas;
|
||||
typedef FunctionArrayIndex<IndexIdentity, NameIndexOf> FunctionIndexOf;
|
||||
typedef FunctionArrayIndex<IndexCount, NameCountEqual> FunctionCountEqual;
|
||||
typedef FunctionArrayIndex<IndexCount, NameCountEqual> FunctionCountEqual;
|
||||
|
||||
using FunctionEmptyArrayUInt8 = FunctionEmptyArray<DataTypeUInt8>;
|
||||
using FunctionEmptyArrayUInt16 = FunctionEmptyArray<DataTypeUInt16>;
|
||||
|
@ -2,10 +2,13 @@
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/Columns/ColumnVector.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
#include <DB/Functions/IFunction.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 ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets,
|
||||
const String & b,
|
||||
@ -185,7 +264,7 @@ struct StringIfImpl
|
||||
|
||||
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;
|
||||
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(
|
||||
const PODArray<UInt8> & cond,
|
||||
const String & a,
|
||||
const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_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();
|
||||
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;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
if (negative != cond[i])
|
||||
{
|
||||
size_t size_to_write = a.size() + 1;
|
||||
c_data.resize(c_data.size() + size_to_write);
|
||||
memcpy(&c_data[c_prev_offset], a.data(), size_to_write);
|
||||
c_prev_offset += size_to_write;
|
||||
size_t size_to_write = a_N;
|
||||
c_data.resize(c_data.size() + size_to_write + 1);
|
||||
memcpy(&c_data[c_prev_offset], &a_data[i * a_N], size_to_write);
|
||||
c_data.back() = 0;
|
||||
c_prev_offset += size_to_write + 1;
|
||||
c_offsets[i] = c_prev_offset;
|
||||
}
|
||||
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);
|
||||
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_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(
|
||||
const PODArray<UInt8> & cond,
|
||||
const String & a, const String & b,
|
||||
@ -495,6 +609,7 @@ public:
|
||||
|
||||
/** Реализация для массивов строк.
|
||||
* NOTE: Код слишком сложный, потому что он работает в внутренностями массивов строк.
|
||||
* NOTE: Массивы из FixedString не поддерживаются.
|
||||
*/
|
||||
struct StringArrayIfImpl
|
||||
{
|
||||
@ -1013,43 +1128,94 @@ private:
|
||||
|
||||
const ColumnString * col_then = typeid_cast<const ColumnString *>(col_then_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_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;
|
||||
block.getByPosition(result).column = col_res;
|
||||
if (col_then_fixed && col_else_fixed)
|
||||
{
|
||||
/// Результат - FixedString.
|
||||
|
||||
ColumnString::Chars_t & res_vec = col_res->getChars();
|
||||
ColumnString::Offsets_t & res_offsets = col_res->getOffsets();
|
||||
if (col_then_fixed->getN() != col_else_fixed->getN())
|
||||
throw Exception("FixedString columns as 'then' and 'else' arguments of function 'if' has different sizes", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (col_then && col_else)
|
||||
StringIfImpl::vector_vector(
|
||||
size_t N = col_then_fixed->getN();
|
||||
|
||||
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(),
|
||||
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);
|
||||
col_then_fixed->getChars(),
|
||||
col_else_fixed->getChars(),
|
||||
N,
|
||||
res_vec);
|
||||
}
|
||||
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;
|
||||
}
|
||||
@ -1151,9 +1317,29 @@ public:
|
||||
}
|
||||
else if (arguments[1]->getName() != arguments[2]->getName())
|
||||
{
|
||||
throw Exception("Incompatible second and third arguments for function " + getName() + ": "
|
||||
+ arguments[1]->getName() + " and " + arguments[2]->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
const DataTypeString * type_string1 = typeid_cast<const DataTypeString *>(arguments[1].get());
|
||||
const DataTypeString * type_string2 = typeid_cast<const DataTypeString *>(arguments[2].get());
|
||||
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];
|
||||
|
@ -1018,7 +1018,7 @@ public:
|
||||
* Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
* Для неконстантных столбцов arguments[i].column = nullptr.
|
||||
*/
|
||||
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments,
|
||||
void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
|
||||
DataTypePtr & out_return_type,
|
||||
std::vector<ExpressionAction> & out_prerequisites)
|
||||
{
|
||||
@ -1099,7 +1099,7 @@ public:
|
||||
|
||||
private:
|
||||
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))
|
||||
return false;
|
||||
@ -1113,7 +1113,7 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t getSize(const ColumnWithNameAndType & column)
|
||||
size_t getSize(const ColumnWithTypeAndName & column)
|
||||
{
|
||||
size_t res;
|
||||
if (getSizeTyped<UInt8>(column, res) ||
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <DB/Dictionaries/FlatDictionary.h>
|
||||
#include <DB/Dictionaries/HashedDictionary.h>
|
||||
#include <DB/Dictionaries/CacheDictionary.h>
|
||||
#include <DB/Dictionaries/RangeHashedDictionary.h>
|
||||
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
|
||||
@ -755,10 +756,10 @@ public:
|
||||
private:
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 3)
|
||||
if (arguments.size() != 3 && arguments.size() != 4)
|
||||
throw Exception{
|
||||
"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
|
||||
};
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
@ -806,7 +816,8 @@ private:
|
||||
|
||||
if (!executeDispatch<FlatDictionary>(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{
|
||||
"Unsupported dictionary type " + dict_ptr->getTypeName(),
|
||||
ErrorCodes::UNKNOWN_TYPE
|
||||
@ -814,13 +825,20 @@ private:
|
||||
}
|
||||
|
||||
template <typename DictionaryType>
|
||||
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const IDictionary * const dictionary)
|
||||
bool executeDispatch(
|
||||
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() != 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());
|
||||
if (!attr_name_col)
|
||||
throw Exception{
|
||||
@ -835,13 +853,16 @@ private:
|
||||
{
|
||||
const auto out = new ColumnString;
|
||||
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))
|
||||
{
|
||||
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>{
|
||||
id_col->size(),
|
||||
dictionary->getString(attr_name, id_col->getData())
|
||||
id_col->size(), out->getDataAtWithTerminatingZero(0).toString()
|
||||
};
|
||||
}
|
||||
else
|
||||
@ -855,6 +876,109 @@ private:
|
||||
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;
|
||||
};
|
||||
|
||||
@ -863,14 +987,20 @@ template <typename DataType> struct DictGetTraits;
|
||||
#define DECLARE_DICT_GET_TRAITS(TYPE, DATA_TYPE) \
|
||||
template <> struct DictGetTraits<DATA_TYPE>\
|
||||
{\
|
||||
static TYPE get(const IDictionary * const dict, const std::string & name, const IDictionary::id_t id)\
|
||||
{\
|
||||
return dict->get##TYPE(name, id);\
|
||||
}\
|
||||
static void get(const IDictionary * const dict, const std::string & name, const PODArray<IDictionary::id_t> & ids, PODArray<TYPE> & out)\
|
||||
template <typename DictionaryType>\
|
||||
static void get(\
|
||||
const DictionaryType * const dict, const std::string & name, const PODArray<UInt64> & ids,\
|
||||
PODArray<TYPE> & 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(UInt16, DataTypeUInt16)
|
||||
@ -906,10 +1036,10 @@ public:
|
||||
private:
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 3)
|
||||
if (arguments.size() != 3 && arguments.size() != 4)
|
||||
throw Exception{
|
||||
"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
|
||||
};
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
@ -957,7 +1096,8 @@ private:
|
||||
|
||||
if (!executeDispatch<FlatDictionary>(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{
|
||||
"Unsupported dictionary type " + dict_ptr->getTypeName(),
|
||||
ErrorCodes::UNKNOWN_TYPE
|
||||
@ -966,12 +1106,19 @@ private:
|
||||
|
||||
template <typename DictionaryType>
|
||||
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);
|
||||
if (!dict)
|
||||
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());
|
||||
if (!attr_name_col)
|
||||
throw Exception{
|
||||
@ -992,14 +1139,15 @@ private:
|
||||
const auto size = ids.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))
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<Type>{
|
||||
id_col->size(),
|
||||
DictGetTraits<DataType>::get(dictionary, attr_name, id_col->getData())
|
||||
};
|
||||
const PODArray<UInt64> ids(1, id_col->getData());
|
||||
PODArray<Type> data(1);
|
||||
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
|
||||
|
||||
block.getByPosition(result).column = new ColumnConst<Type>{id_col->size(), data.front()};
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1012,6 +1160,120 @@ private:
|
||||
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;
|
||||
};
|
||||
|
||||
@ -1090,12 +1352,6 @@ private:
|
||||
auto dict = dictionaries.getDictionary(dict_name_col->getData());
|
||||
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) &&
|
||||
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
|
||||
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
|
||||
@ -1107,16 +1363,19 @@ private:
|
||||
|
||||
template <typename DictionaryType>
|
||||
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);
|
||||
if (!dict)
|
||||
return false;
|
||||
|
||||
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();
|
||||
if (!dict->hasHierarchy())
|
||||
throw Exception{
|
||||
"Dictionary does not have a hierarchy",
|
||||
ErrorCodes::UNSUPPORTED_METHOD
|
||||
};
|
||||
|
||||
const auto get_hierarchies = [&] (const PODArray<UInt64> & in, PODArray<UInt64> & out, PODArray<UInt64> & offsets) {
|
||||
const auto size = in.size();
|
||||
|
||||
/// copy of `in` array
|
||||
@ -1151,18 +1410,12 @@ private:
|
||||
break;
|
||||
|
||||
/// 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
|
||||
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);
|
||||
offsets.resize(size);
|
||||
|
||||
@ -1172,21 +1425,29 @@ private:
|
||||
out.insert_assume_reserved(std::begin(ids), std::end(ids));
|
||||
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))
|
||||
{
|
||||
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();
|
||||
while (cur)
|
||||
{
|
||||
res.push_back(cur);
|
||||
cur = dictionary->toParent(cur);
|
||||
}
|
||||
get_hierarchies(in, backend->getData(), array->getOffsets());
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstArray{
|
||||
id_col->size(),
|
||||
res,
|
||||
(*array)[0].get<Array>(),
|
||||
new DataTypeArray{new DataTypeUInt64}
|
||||
};
|
||||
}
|
||||
@ -1280,12 +1541,6 @@ private:
|
||||
auto dict = dictionaries.getDictionary(dict_name_col->getData());
|
||||
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) &&
|
||||
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
|
||||
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
|
||||
@ -1297,19 +1552,25 @@ private:
|
||||
|
||||
template <typename DictionaryType>
|
||||
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);
|
||||
if (!dict)
|
||||
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 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))
|
||||
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))
|
||||
execute(block, result, dictionary, child_id_col, ancestor_id_col_untyped);
|
||||
execute(block, result, dict, child_id_col, ancestor_id_col_untyped);
|
||||
else
|
||||
throw Exception{
|
||||
"Illegal column " + child_id_col_untyped->getName()
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <openssl/md5.h>
|
||||
#include <openssl/sha.h>
|
||||
#include <city.h>
|
||||
#include <farmhash.h>
|
||||
#include <metrohash.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:
|
||||
static constexpr auto name = "cityHash64";
|
||||
static IFunction * create(const Context & context) { return new FunctionCityHash64; };
|
||||
static constexpr auto name = Impl::name;
|
||||
static IFunction * create(const Context & context) { return new FunctionNeighbourhoodHash64; };
|
||||
|
||||
private:
|
||||
template <typename FromType, bool first>
|
||||
@ -417,12 +420,12 @@ private:
|
||||
if (first)
|
||||
vec_to[i] = h;
|
||||
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))
|
||||
{
|
||||
UInt64 hash = IntHash64Impl::apply(toInteger(col_from->getData()));
|
||||
const UInt64 hash = IntHash64Impl::apply(toInteger(col_from->getData()));
|
||||
size_t size = vec_to.size();
|
||||
if (first)
|
||||
{
|
||||
@ -431,7 +434,7 @@ private:
|
||||
else
|
||||
{
|
||||
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
|
||||
@ -451,13 +454,13 @@ private:
|
||||
|
||||
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]]),
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
|
||||
if (first)
|
||||
vec_to[i] = h;
|
||||
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))
|
||||
@ -467,17 +470,17 @@ private:
|
||||
size_t size = data.size() / n;
|
||||
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)
|
||||
vec_to[i] = h;
|
||||
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))
|
||||
{
|
||||
UInt64 hash = CityHash64(col_from->getData().data(), col_from->getData().size());
|
||||
size_t size = vec_to.size();
|
||||
const UInt64 hash = Impl::Hash64(col_from->getData().data(), col_from->getData().size());
|
||||
const size_t size = vec_to.size();
|
||||
if (first)
|
||||
{
|
||||
vec_to.assign(size, hash);
|
||||
@ -486,7 +489,7 @@ private:
|
||||
{
|
||||
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 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);
|
||||
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)
|
||||
{
|
||||
size_t begin = i == 0 ? 0 : offsets[i - 1];
|
||||
size_t end = offsets[i];
|
||||
const size_t begin = i == 0 ? 0 : offsets[i - 1];
|
||||
const size_t end = offsets[i];
|
||||
|
||||
UInt64 h = IntHash64Impl::apply(end - begin);
|
||||
if (first)
|
||||
vec_to[i] = h;
|
||||
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)
|
||||
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))
|
||||
@ -592,7 +595,7 @@ public:
|
||||
|
||||
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 IColumn * icolumn = &*column.column;
|
||||
|
||||
@ -702,7 +705,7 @@ public:
|
||||
"Illegal type " + first_arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
|
||||
|
||||
if (arg_count == 2)
|
||||
{
|
||||
const auto second_arg = arguments.back().get();
|
||||
@ -816,18 +819,57 @@ private:
|
||||
|
||||
struct NameHalfMD5 { static constexpr auto name = "halfMD5"; };
|
||||
struct NameSipHash64 { static constexpr auto name = "sipHash64"; };
|
||||
struct NameCityHash64 { static constexpr auto name = "cityHash64"; };
|
||||
struct NameIntHash32 { static constexpr auto name = "intHash32"; };
|
||||
struct NameIntHash64 { static constexpr auto name = "intHash64"; };
|
||||
|
||||
typedef FunctionStringHash64<HalfMD5Impl, NameHalfMD5> FunctionHalfMD5;
|
||||
typedef FunctionStringHash64<SipHash64Impl, NameSipHash64> FunctionSipHash64;
|
||||
typedef FunctionIntHash<IntHash32Impl, NameIntHash32> FunctionIntHash32;
|
||||
typedef FunctionIntHash<IntHash64Impl, NameIntHash64> FunctionIntHash64;
|
||||
typedef FunctionStringHashFixedString<MD5Impl> FunctionMD5;
|
||||
typedef FunctionStringHashFixedString<SHA1Impl> FunctionSHA1;
|
||||
typedef FunctionStringHashFixedString<SHA224Impl> FunctionSHA224;
|
||||
typedef FunctionStringHashFixedString<SHA256Impl> FunctionSHA256;
|
||||
typedef FunctionStringHashFixedString<SipHash128Impl> FunctionSipHash128;
|
||||
struct ImplCityHash64
|
||||
{
|
||||
static constexpr auto name = "cityHash64";
|
||||
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) { return CityHash64(s, len); }
|
||||
};
|
||||
|
||||
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>;
|
||||
|
||||
}
|
||||
|
@ -586,7 +586,7 @@ public:
|
||||
arguments[0] = new DataTypeExpression(nested_types);
|
||||
}
|
||||
|
||||
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments,
|
||||
void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
|
||||
DataTypePtr & out_return_type,
|
||||
ExpressionActions::Actions & out_prerequisites)
|
||||
{
|
||||
@ -725,7 +725,7 @@ public:
|
||||
column_first_array = column_array;
|
||||
}
|
||||
|
||||
temp_block.insert(ColumnWithNameAndType(
|
||||
temp_block.insert(ColumnWithTypeAndName(
|
||||
column_array->getDataPtr(),
|
||||
argument_type,
|
||||
argument_name));
|
||||
@ -745,7 +745,7 @@ public:
|
||||
if (argument_names.count(name))
|
||||
continue;
|
||||
|
||||
ColumnWithNameAndType replicated_column = block.getByPosition(prerequisites[prerequisite_index]);
|
||||
ColumnWithTypeAndName replicated_column = block.getByPosition(prerequisites[prerequisite_index]);
|
||||
|
||||
replicated_column.name = name;
|
||||
replicated_column.column = typeid_cast<ColumnArray &>(*replicated_column.column).getDataPtr();
|
||||
|
@ -453,7 +453,7 @@ public:
|
||||
return name;
|
||||
}
|
||||
|
||||
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments,
|
||||
void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
|
||||
DataTypePtr & out_return_type,
|
||||
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
|
||||
{
|
||||
public:
|
||||
|
@ -132,7 +132,8 @@ public:
|
||||
const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
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);
|
||||
|
||||
const String & sep_str = col->getData();
|
||||
@ -198,7 +199,8 @@ public:
|
||||
const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
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);
|
||||
|
||||
sep = col->getData();
|
||||
@ -264,7 +266,8 @@ public:
|
||||
const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[1]).column);
|
||||
|
||||
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);
|
||||
|
||||
re = Regexps::get<false, false>(col->getData());
|
||||
|
@ -404,31 +404,36 @@ struct ExtractURLParameterImpl
|
||||
{
|
||||
size_t cur_offset = offsets[i];
|
||||
|
||||
const char * str = reinterpret_cast<const char *>(&data[prev_offset]);
|
||||
|
||||
const char * pos = nullptr;
|
||||
|
||||
do
|
||||
const char * begin = strchr(str, '?');
|
||||
if (begin != nullptr)
|
||||
{
|
||||
const char * str = reinterpret_cast<const char *>(&data[prev_offset]);
|
||||
|
||||
const char * begin = strchr(str, '?');
|
||||
if (begin == nullptr)
|
||||
break;
|
||||
|
||||
pos = strstr(begin + 1, param_str);
|
||||
if (pos == nullptr)
|
||||
break;
|
||||
if (pos != begin + 1 && *(pos - 1) != ';' && *(pos - 1) != '&')
|
||||
pos = begin + 1;
|
||||
while (true)
|
||||
{
|
||||
pos = nullptr;
|
||||
break;
|
||||
}
|
||||
pos = strstr(pos, param_str);
|
||||
|
||||
pos += param_len;
|
||||
} while (false);
|
||||
if (pos == nullptr)
|
||||
break;
|
||||
|
||||
if (pos[-1] != '?' && pos[-1] != '&')
|
||||
{
|
||||
pos += param_len;
|
||||
continue;
|
||||
}
|
||||
else
|
||||
{
|
||||
pos += param_len;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (pos != nullptr)
|
||||
{
|
||||
const char * end = strpbrk(pos, "&;#");
|
||||
const char * end = strpbrk(pos, "&#");
|
||||
if (end == nullptr)
|
||||
end = pos + strlen(pos);
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <DB/Core/Names.h>
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/Core/ColumnNumbers.h>
|
||||
#include <DB/Core/ColumnsWithNameAndType.h>
|
||||
#include <DB/Core/ColumnsWithTypeAndName.h>
|
||||
#include <DB/DataTypes/IDataType.h>
|
||||
|
||||
|
||||
@ -54,7 +54,7 @@ public:
|
||||
* Осмысленные типы элементов в out_prerequisites: APPLY_FUNCTION, ADD_COLUMN.
|
||||
*/
|
||||
virtual void getReturnTypeAndPrerequisites(
|
||||
const ColumnsWithNameAndType & arguments,
|
||||
const ColumnsWithTypeAndName & arguments,
|
||||
DataTypePtr & out_return_type,
|
||||
std::vector<ExpressionAction> & out_prerequisites)
|
||||
{
|
||||
|
@ -85,8 +85,7 @@ inline void writeBoolText(bool x, WriteBuffer & buf)
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
void writeFloatText(T x, WriteBuffer & buf)
|
||||
inline void writeFloatText(double x, WriteBuffer & buf)
|
||||
{
|
||||
char tmp[25];
|
||||
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);
|
||||
|
||||
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());
|
||||
}
|
||||
|
@ -374,8 +374,20 @@ struct AggregationMethodConcat
|
||||
/// См. функцию extractKeysAndPlaceInPoolContiguous.
|
||||
const StringRef * key_refs = reinterpret_cast<const StringRef *>(value.first.data + value.first.size);
|
||||
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
key_columns[i]->insertDataWithTerminatingZero(key_refs[i].data, key_refs[i].size);
|
||||
if (unlikely(0 == value.first.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);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -41,6 +41,7 @@ class Macros;
|
||||
class Progress;
|
||||
class Clusters;
|
||||
class QueryLog;
|
||||
struct MergeTreeSettings;
|
||||
|
||||
|
||||
/// имя таблицы -> таблица
|
||||
@ -92,7 +93,6 @@ private:
|
||||
std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
|
||||
String current_database; /// Текущая БД.
|
||||
String current_query_id; /// Id текущего запроса.
|
||||
NamesAndTypesList columns; /// Столбцы текущей обрабатываемой таблицы.
|
||||
Settings settings; /// Настройки выполнения запроса.
|
||||
using ProgressCallback = std::function<void(const Progress & progress)>;
|
||||
ProgressCallback progress_callback; /// Колбек для отслеживания прогресса выполнения запроса.
|
||||
@ -207,11 +207,6 @@ public:
|
||||
const Databases & getDatabases() const;
|
||||
Databases & getDatabases();
|
||||
|
||||
/// При работе со списком столбцов, используйте локальный контекст, чтобы никто больше его не менял.
|
||||
const NamesAndTypesList & getColumns() const { return columns; }
|
||||
NamesAndTypesList & getColumns() { return columns; }
|
||||
void setColumns(const NamesAndTypesList & columns_) { columns = columns_; }
|
||||
|
||||
Context & getSessionContext();
|
||||
Context & getGlobalContext();
|
||||
|
||||
@ -266,8 +261,8 @@ public:
|
||||
Poco::SharedPtr<Clusters> getClusters() const;
|
||||
|
||||
Compiler & getCompiler();
|
||||
|
||||
QueryLog & getQueryLog();
|
||||
const MergeTreeSettings & getMergeTreeSettings();
|
||||
|
||||
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
|
||||
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <DB/DataStreams/IBlockInputStream.h>
|
||||
#include <DB/Interpreters/Settings.h>
|
||||
#include <DB/Core/Names.h>
|
||||
#include <DB/Core/ColumnWithNameAndType.h>
|
||||
#include <DB/Core/ColumnWithTypeAndName.h>
|
||||
#include <DB/Core/Block.h>
|
||||
|
||||
#include <unordered_set>
|
||||
@ -66,6 +66,7 @@ public:
|
||||
|
||||
/// Для ARRAY_JOIN
|
||||
NameSet array_joined_columns;
|
||||
bool array_join_is_left;
|
||||
|
||||
/// Для JOIN
|
||||
const Join * join = nullptr;
|
||||
@ -77,7 +78,7 @@ public:
|
||||
/// Если 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;
|
||||
a.type = ADD_COLUMN;
|
||||
@ -122,13 +123,14 @@ public:
|
||||
return a;
|
||||
}
|
||||
|
||||
static ExpressionAction arrayJoin(const NameSet & array_joined_columns)
|
||||
static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left)
|
||||
{
|
||||
if (array_joined_columns.empty())
|
||||
throw Exception("No arrays to join", ErrorCodes::LOGICAL_ERROR);
|
||||
ExpressionAction a;
|
||||
a.type = ARRAY_JOIN;
|
||||
a.array_joined_columns = array_joined_columns;
|
||||
a.array_join_is_left = array_join_is_left;
|
||||
return a;
|
||||
}
|
||||
|
||||
@ -168,11 +170,11 @@ public:
|
||||
: input_columns(input_columns_), settings(settings_)
|
||||
{
|
||||
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_ могут содержаться сами столбцы.
|
||||
ExpressionActions(const ColumnsWithNameAndType & input_columns_, const Settings & settings_)
|
||||
ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Settings & settings_)
|
||||
: settings(settings_)
|
||||
{
|
||||
for (const auto & input_elem : input_columns_)
|
||||
@ -185,7 +187,7 @@ public:
|
||||
/// Добавить входной столбец.
|
||||
/// Название столбца не должно совпадать с названиями промежуточных столбцов, возникающих при вычислении выражения.
|
||||
/// В выражении не должно быть действий PROJECT.
|
||||
void addInput(const ColumnWithNameAndType & column);
|
||||
void addInput(const ColumnWithTypeAndName & column);
|
||||
void addInput(const NameAndTypePair & column);
|
||||
|
||||
void add(const ExpressionAction & action);
|
||||
|
@ -46,17 +46,18 @@ typedef std::unordered_map<String, SubqueryForSet> SubqueriesForSets;
|
||||
class ExpressionAnalyzer : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
ExpressionAnalyzer(const ASTPtr & ast_, const Context & context_, StoragePtr storage_, size_t subquery_depth_ = 0, bool do_global_ = false)
|
||||
: ast(ast_), context(context_), settings(context.getSettings()),
|
||||
subquery_depth(subquery_depth_), columns(context.getColumns()), storage(storage_ ? storage_ : getTable()), do_global(do_global_)
|
||||
{
|
||||
init();
|
||||
}
|
||||
|
||||
/// 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()),
|
||||
subquery_depth(subquery_depth_), columns(columns_), storage(getTable()), do_global(do_global_)
|
||||
ExpressionAnalyzer(
|
||||
const ASTPtr & ast_,
|
||||
const Context & context_,
|
||||
StoragePtr storage_,
|
||||
const NamesAndTypesList & columns_,
|
||||
size_t subquery_depth_ = 0,
|
||||
bool do_global_ = false)
|
||||
:
|
||||
ast(ast_), context(context_), settings(context.getSettings()),
|
||||
subquery_depth(subquery_depth_), columns(columns_),
|
||||
storage(storage_ ? storage_ : getTable()),
|
||||
do_global(do_global_)
|
||||
{
|
||||
init();
|
||||
}
|
||||
@ -225,6 +226,10 @@ private:
|
||||
/// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn.
|
||||
void makeSet(ASTFunction * node, const Block & sample_block);
|
||||
|
||||
/// Замена скалярных подзапросов на значения-константы.
|
||||
void executeScalarSubqueries();
|
||||
void executeScalarSubqueriesImpl(ASTPtr & ast);
|
||||
|
||||
/// Находит глобальные подзапросы в секциях GLOBAL IN/JOIN. Заполняет external_tables.
|
||||
void initGlobalSubqueriesAndExternalTables();
|
||||
void initGlobalSubqueries(ASTPtr & ast);
|
||||
|
@ -42,7 +42,7 @@ private:
|
||||
|
||||
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
|
||||
{
|
||||
dictionary_ptr_t dict;
|
||||
@ -52,7 +52,7 @@ private:
|
||||
|
||||
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::uint64_t error_count;
|
||||
};
|
||||
@ -107,7 +107,7 @@ public:
|
||||
reloading_thread.join();
|
||||
}
|
||||
|
||||
MultiVersion<IDictionary>::Version getDictionary(const std::string & name) const;
|
||||
MultiVersion<IDictionaryBase>::Version getDictionary(const std::string & name) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -24,8 +24,7 @@ public:
|
||||
*/
|
||||
BlockIO execute() override
|
||||
{
|
||||
executeImpl(false);
|
||||
return {};
|
||||
return executeImpl(false);
|
||||
}
|
||||
|
||||
/** assume_metadata_exists - не проверять наличие файла с метаданными и не создавать его
|
||||
@ -45,7 +44,7 @@ public:
|
||||
const ColumnDefaults & column_defaults);
|
||||
|
||||
private:
|
||||
void executeImpl(bool assume_metadata_exists);
|
||||
BlockIO executeImpl(bool assume_metadata_exists);
|
||||
|
||||
/// AST в список столбцов с типами. Столбцы типа Nested развернуты в список настоящих столбцов.
|
||||
using ColumnsAndDefaults = std::pair<NamesAndTypesList, ColumnDefaults>;
|
||||
|
@ -44,7 +44,7 @@ private:
|
||||
{
|
||||
Block block;
|
||||
|
||||
ColumnWithNameAndType col;
|
||||
ColumnWithTypeAndName col;
|
||||
col.name = "name";
|
||||
col.type = new DataTypeString;
|
||||
col.column = col.type->createColumn();
|
||||
@ -78,10 +78,10 @@ private:
|
||||
column_defaults = table->column_defaults;
|
||||
}
|
||||
|
||||
ColumnWithNameAndType name_column{new ColumnString, new DataTypeString, "name"};
|
||||
ColumnWithNameAndType type_column{new ColumnString, new DataTypeString, "type" };
|
||||
ColumnWithNameAndType default_type_column{new ColumnString, new DataTypeString, "default_type" };
|
||||
ColumnWithNameAndType default_expression_column{new ColumnString, new DataTypeString, "default_expression" };;
|
||||
ColumnWithTypeAndName name_column{new ColumnString, new DataTypeString, "name"};
|
||||
ColumnWithTypeAndName type_column{new ColumnString, new DataTypeString, "type" };
|
||||
ColumnWithTypeAndName default_type_column{new ColumnString, new DataTypeString, "default_type" };
|
||||
ColumnWithTypeAndName default_expression_column{new ColumnString, new DataTypeString, "default_expression" };;
|
||||
|
||||
for (const auto column : columns)
|
||||
{
|
||||
|
@ -39,7 +39,7 @@ private:
|
||||
|
||||
Block getSampleBlock()
|
||||
{
|
||||
ColumnWithNameAndType col;
|
||||
ColumnWithTypeAndName col;
|
||||
col.name = "result";
|
||||
col.type = new DataTypeUInt8;
|
||||
col.column = col.type->createColumn();
|
||||
@ -56,7 +56,7 @@ private:
|
||||
|
||||
bool res = context.isTableExist(ast.database, ast.table);
|
||||
|
||||
ColumnWithNameAndType col;
|
||||
ColumnWithTypeAndName col;
|
||||
col.name = "result";
|
||||
col.type = new DataTypeUInt8;
|
||||
col.column = new ColumnConstUInt8(1, res);
|
||||
|
@ -33,7 +33,7 @@ public:
|
||||
* - удалить из запроса все столбцы кроме указанных - используется для удаления ненужных столбцов из подзапросов.
|
||||
*
|
||||
* table_column_names
|
||||
* - поместить в контекст в качестве известных столбцов только указанные столбцы, а не все столбцы таблицы.
|
||||
* - список доступных столбцов таблицы.
|
||||
* Используется, например, совместно с указанием input.
|
||||
*/
|
||||
|
||||
@ -42,8 +42,7 @@ public:
|
||||
const Context & context_,
|
||||
QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete,
|
||||
size_t subquery_depth_ = 0,
|
||||
BlockInputStreamPtr input = nullptr,
|
||||
bool is_union_all_head_ = true);
|
||||
BlockInputStreamPtr input = nullptr);
|
||||
|
||||
InterpreterSelectQuery(
|
||||
ASTPtr query_ptr_,
|
||||
@ -57,7 +56,7 @@ public:
|
||||
ASTPtr query_ptr_,
|
||||
const Context & context_,
|
||||
const Names & required_column_names,
|
||||
const NamesAndTypesList & table_column_names,
|
||||
const NamesAndTypesList & table_column_names_,
|
||||
QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete,
|
||||
size_t subquery_depth_ = 0,
|
||||
BlockInputStreamPtr input = nullptr);
|
||||
@ -76,9 +75,23 @@ public:
|
||||
DataTypes getReturnTypes();
|
||||
Block getSampleBlock();
|
||||
|
||||
static Block getSampleBlock(
|
||||
ASTPtr query_ptr_,
|
||||
const Context & context_);
|
||||
|
||||
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();
|
||||
|
||||
/// Выполнить один запрос SELECT из цепочки UNION ALL.
|
||||
@ -96,7 +109,7 @@ private:
|
||||
// Переименовать столбцы каждого запроса цепочки UNION ALL в такие же имена, как в первом запросе.
|
||||
void renameColumns();
|
||||
|
||||
/** Из какой таблицы читать. При JOIN, возвращается "левая" таблицы.
|
||||
/** Из какой таблицы читать. При JOIN, возвращается "левая" таблица.
|
||||
*/
|
||||
void getDatabaseAndTableNames(String & database_name, String & table_name);
|
||||
|
||||
@ -107,22 +120,43 @@ private:
|
||||
/// Разные стадии выполнения запроса.
|
||||
|
||||
/// Вынимает данные из таблицы. Возвращает стадию, до которой запрос был обработан в 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();
|
||||
|
||||
@ -143,12 +177,28 @@ private:
|
||||
QueryProcessingStage::Enum to_stage;
|
||||
size_t subquery_depth;
|
||||
std::unique_ptr<ExpressionAnalyzer> query_analyzer;
|
||||
NamesAndTypesList table_column_names;
|
||||
|
||||
/** Потоки данных.
|
||||
* Исходные потоки данных получаются в функции executeFetchColumns.
|
||||
* Затем они преобразуются (оборачиваются в другие потоки) с помощью функций execute*,
|
||||
* чтобы получить целый конвейер выполнения запроса.
|
||||
*/
|
||||
BlockInputStreams streams;
|
||||
|
||||
/** При выполнении FULL или RIGHT JOIN, здесь будет поток данных, из которого можно прочитать "неприсоединённые" строки.
|
||||
* Он имеет особое значение, так как чтение из него должно осуществляться после чтения из основных потоков.
|
||||
* Он подклеивается к основным потокам в UnionBlockInputStream или ParallelAggregatingBlockInputStream.
|
||||
*/
|
||||
BlockInputStreamPtr stream_with_non_joined_data;
|
||||
|
||||
/// Являемся ли мы первым запросом SELECT цепочки 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;
|
||||
|
||||
/// Таблица, откуда читать данные, если не подзапрос.
|
||||
|
@ -40,7 +40,7 @@ private:
|
||||
|
||||
Block getSampleBlock()
|
||||
{
|
||||
ColumnWithNameAndType col;
|
||||
ColumnWithTypeAndName col;
|
||||
col.name = "statement";
|
||||
col.type = new DataTypeString;
|
||||
col.column = col.type->createColumn();
|
||||
@ -59,7 +59,7 @@ private:
|
||||
formatAST(*context.getCreateQuery(ast.database, ast.table), stream, 0, false, true);
|
||||
String res = stream.str();
|
||||
|
||||
ColumnWithNameAndType col;
|
||||
ColumnWithTypeAndName col;
|
||||
col.name = "statement";
|
||||
col.type = new DataTypeString;
|
||||
col.column = new ColumnConstString(1, res);
|
||||
|
@ -18,8 +18,9 @@ namespace DB
|
||||
|
||||
/** Структура данных для реализации JOIN-а.
|
||||
* По сути, хэш-таблица: ключи -> строки присоединяемой таблицы.
|
||||
* Исключение - CROSS JOIN, где вместо хэш-таблицы просто набор блоков без ключей.
|
||||
*
|
||||
* JOIN-ы бывают восьми типов: ANY/ALL x LEFT/INNER/RIGHT/FULL.
|
||||
* JOIN-ы бывают девяти типов: ANY/ALL × LEFT/INNER/RIGHT/FULL, а также CROSS.
|
||||
*
|
||||
* Если указано ANY - выбрать из "правой" таблицы только одну, первую попавшуюся строку, даже если там более одной соответствующей строки.
|
||||
* Если указано ALL - обычный вариант JOIN-а, при котором строки могут размножаться по числу соответствующих строк "правой" таблицы.
|
||||
@ -213,6 +214,7 @@ private:
|
||||
KEY_64,
|
||||
KEY_STRING,
|
||||
HASHED,
|
||||
CROSS,
|
||||
};
|
||||
|
||||
Type type = Type::EMPTY;
|
||||
@ -249,6 +251,8 @@ private:
|
||||
template <ASTJoin::Kind KIND, ASTJoin::Strictness STRICTNESS, typename Maps>
|
||||
void joinBlockImpl(Block & block, const Maps & maps) const;
|
||||
|
||||
void joinBlockImplCross(Block & block) const;
|
||||
|
||||
/// Проверить не превышены ли допустимые размеры множества
|
||||
bool checkSizeLimits() const;
|
||||
|
||||
|
@ -39,11 +39,16 @@ inline void evaluateMissingDefaults(Block & block,
|
||||
* we are going to operate on a copy instead of the original block */
|
||||
Block copy_block{block};
|
||||
/// evaluate default values for defaulted columns
|
||||
ExpressionAnalyzer{default_expr_list, context, required_columns}.getActions(true)->execute(copy_block);
|
||||
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())
|
||||
{
|
||||
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));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -22,7 +22,7 @@ namespace DB
|
||||
/** pass a dummy column name because ExpressionAnalyzer
|
||||
* does not work with no columns so far. */
|
||||
ExpressionAnalyzer{
|
||||
expr, context,
|
||||
expr, context, {},
|
||||
{ { "", new DataTypeString } }
|
||||
}.getActions(false)->execute(block);
|
||||
|
||||
|
@ -32,7 +32,8 @@ public:
|
||||
Inner, /// Оставить только записи, для которых в "правой" таблице есть соответствующая.
|
||||
Left, /// Если в "правой" таблице нет соответствующих записей, заполнить столбцы значениями "по-умолчанию".
|
||||
Right,
|
||||
Full
|
||||
Full,
|
||||
Cross /// Прямое произведение. strictness и using_expr_list не используются.
|
||||
};
|
||||
|
||||
Locality locality = Local;
|
||||
@ -61,7 +62,8 @@ public:
|
||||
kind == Inner ? "Inner"
|
||||
: (kind == Left ? "Left"
|
||||
: (kind == Right ? "Right"
|
||||
: "Full")), wb);
|
||||
: (kind == Full ? "Full"
|
||||
: "Cross"))), wb);
|
||||
|
||||
writeString("Join", wb);
|
||||
}
|
||||
|
@ -32,16 +32,25 @@ public:
|
||||
/// Переписывает select_expression_list, чтобы вернуть только необходимые столбцы в правильном порядке.
|
||||
void rewriteSelectExpressionList(const Names & column_names);
|
||||
|
||||
bool isUnionAllHead() const { return prev_union_all.isNull() && !next_union_all.isNull(); }
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
/// Получить глубокую копию дерева первого запроса SELECT.
|
||||
ASTPtr cloneFirstSelect() const;
|
||||
|
||||
/// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
|
||||
const IAST * getFormat() const override;
|
||||
|
||||
private:
|
||||
ASTPtr cloneImpl(bool traverse_union_all) const;
|
||||
|
||||
public:
|
||||
bool distinct = false;
|
||||
ASTPtr select_expression_list;
|
||||
ASTPtr database;
|
||||
ASTPtr table; /// Идентификатор, табличная функция или подзапрос (рекурсивно ASTSelectQuery)
|
||||
bool array_join_is_left = false; /// LEFT ARRAY JOIN
|
||||
ASTPtr array_join_expression_list; /// ARRAY JOIN
|
||||
ASTPtr join; /// Обычный (не ARRAY) JOIN.
|
||||
bool final = false;
|
||||
@ -55,7 +64,10 @@ public:
|
||||
ASTPtr limit_offset;
|
||||
ASTPtr limit_length;
|
||||
ASTPtr settings;
|
||||
ASTPtr next_union_all; /// Следующий запрос SELECT в цепочке UNION ALL, если такой есть
|
||||
/// Предыдущий запрос SELECT в цепочке UNION ALL (не вставляется в children и не клонируется)
|
||||
ASTPtr prev_union_all;
|
||||
/// Следующий запрос SELECT в цепочке UNION ALL, если такой есть
|
||||
ASTPtr next_union_all;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <DB/DataTypes/IDataType.h>
|
||||
|
||||
#include <DB/Parsers/IAST.h>
|
||||
#include <DB/Parsers/ASTWithAlias.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -11,12 +11,12 @@ namespace DB
|
||||
|
||||
/** Подзарос SELECT
|
||||
*/
|
||||
class ASTSubquery : public IAST
|
||||
class ASTSubquery : public ASTWithAlias
|
||||
{
|
||||
public:
|
||||
ASTSubquery() = default;
|
||||
ASTSubquery(const StringRange range_) : IAST(range_) {}
|
||||
|
||||
ASTSubquery(const StringRange range_) : ASTWithAlias(range_) {}
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const override { return "Subquery"; }
|
||||
|
||||
|
@ -305,7 +305,7 @@ protected:
|
||||
size_t rows = 0;
|
||||
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)
|
||||
continue;
|
||||
column.column = column.column->filter(column_name_set.count(column.name) ? post_filter : pre_filter);
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Storages/MergeTree/ActiveDataPartSet.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeSettings.h>
|
||||
#include <DB/IO/ReadBufferFromString.h>
|
||||
#include <DB/IO/WriteBufferFromFile.h>
|
||||
#include <DB/IO/ReadBufferFromFile.h>
|
||||
@ -69,69 +70,6 @@ namespace DB
|
||||
* - 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
|
||||
{
|
||||
public:
|
||||
@ -841,6 +779,13 @@ public:
|
||||
return it == std::end(column_sizes) ? 0 : it->second;
|
||||
}
|
||||
|
||||
using ColumnSizes = std::unordered_map<std::string, size_t>;
|
||||
ColumnSizes getColumnSizes() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock{data_parts_mutex};
|
||||
return column_sizes;
|
||||
}
|
||||
|
||||
/// Для ATTACH/DETACH/DROP PARTITION.
|
||||
static String getMonthName(const Field & partition);
|
||||
static DayNum_t getMonthDayNum(const Field & partition);
|
||||
@ -872,7 +817,7 @@ private:
|
||||
|
||||
NamesAndTypesListPtr columns;
|
||||
/// Актуальные размеры столбцов в сжатом виде
|
||||
std::unordered_map<std::string, size_t> column_sizes;
|
||||
ColumnSizes column_sizes;
|
||||
|
||||
BrokenPartCallback broken_part_callback;
|
||||
|
||||
|
@ -69,10 +69,11 @@ private:
|
||||
ExpressionActionsPtr prewhere_actions,
|
||||
const String & prewhere_column,
|
||||
const Names & virt_columns,
|
||||
const Settings & settings);
|
||||
const Settings & settings,
|
||||
const Context & context);
|
||||
|
||||
/// Создать выражение "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);
|
||||
};
|
||||
|
@ -92,7 +92,7 @@ public:
|
||||
/// Все столбцы уже есть в блоке. Будем добавлять значения в конец.
|
||||
bool append = res.has(it.name);
|
||||
|
||||
ColumnWithNameAndType column;
|
||||
ColumnWithTypeAndName column;
|
||||
column.name = it.name;
|
||||
column.type = it.type;
|
||||
if (append)
|
||||
@ -247,7 +247,7 @@ private:
|
||||
++right;
|
||||
}
|
||||
|
||||
/// Если правее засечек нет, просто используем DEFAULT_BUFFER_SIZE
|
||||
/// Если правее засечек нет, просто используем max_read_buffer_size
|
||||
if (right >= (*marks).size() || (right + 1 == (*marks).size() &&
|
||||
(*marks)[right].offset_in_compressed_file == (*marks)[all_mark_ranges[i].end].offset_in_compressed_file))
|
||||
{
|
||||
@ -487,7 +487,7 @@ private:
|
||||
OffsetColumns offset_columns;
|
||||
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))
|
||||
{
|
||||
String offsets_name = DataTypeNested::extractNestedTableName(column.name);
|
||||
@ -514,7 +514,7 @@ private:
|
||||
continue;
|
||||
}
|
||||
|
||||
ColumnWithNameAndType column_to_add;
|
||||
ColumnWithTypeAndName column_to_add;
|
||||
column_to_add.name = requested_column.name;
|
||||
column_to_add.type = requested_column.type;
|
||||
|
||||
|
115
dbms/include/DB/Storages/MergeTree/MergeTreeSettings.h
Normal file
115
dbms/include/DB/Storages/MergeTree/MergeTreeSettings.h
Normal 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
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -37,6 +37,8 @@ class MergeTreeWhereOptimizer
|
||||
static constexpr auto and_function_name = "and";
|
||||
static constexpr auto equals_function_name = "equals";
|
||||
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:
|
||||
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
|
||||
* containing said columns should not be moved to PREWHERE at all.
|
||||
* We assume all AS aliases have been expanded prior to using this class */
|
||||
* containing said columns should not be moved to PREWHERE at all.
|
||||
* 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
|
||||
{
|
||||
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
|
||||
if (array_join_function_name == function_ptr->name)
|
||||
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))
|
||||
{
|
||||
|
@ -283,7 +283,7 @@ public:
|
||||
size_t rows = block.rows();
|
||||
|
||||
/// Сначала пишем индекс. Индекс содержит значение Primary Key для каждой index_granularity строки.
|
||||
typedef std::vector<const ColumnWithNameAndType *> PrimaryColumns;
|
||||
typedef std::vector<const ColumnWithTypeAndName *> PrimaryColumns;
|
||||
PrimaryColumns primary_columns;
|
||||
|
||||
for (const auto & descr : storage.getSortDescription())
|
||||
@ -311,7 +311,7 @@ public:
|
||||
/// Теперь пишем данные.
|
||||
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);
|
||||
}
|
||||
|
||||
@ -435,7 +435,7 @@ public:
|
||||
OffsetColumns offset_columns;
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -17,10 +17,10 @@ public:
|
||||
std::string getName() const override { return "ChunkRef"; }
|
||||
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); };
|
||||
bool hasColumn(const String & column_name) const override { return getSource().hasColumn(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); };
|
||||
|
||||
BlockInputStreams read(
|
||||
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);
|
||||
|
||||
/// TODO: может быть, можно просто хранить указатель на родительскую таблицу?
|
||||
StorageChunks & getSource();
|
||||
const StorageChunks & getSource() const;
|
||||
StoragePtr getSource();
|
||||
const StoragePtr getSource() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <DB/Storages/StorageLog.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <statdaemons/CounterInFile.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -89,7 +88,9 @@ private:
|
||||
ChunkIndices chunk_indices;
|
||||
ChunkNumToChunkName chunk_names;
|
||||
|
||||
CounterInFile reference_counter;
|
||||
size_t refcount = 0;
|
||||
std::mutex refcount_mutex;
|
||||
|
||||
Context & context;
|
||||
|
||||
Logger * log;
|
||||
|
@ -43,7 +43,7 @@ public:
|
||||
MergeTreeData::Mode mode_,
|
||||
const String & sign_column_, /// Для Collapsing режима.
|
||||
const Names & columns_to_sum_, /// Для Summing режима.
|
||||
const MergeTreeSettings & settings_ = MergeTreeSettings());
|
||||
const MergeTreeSettings & settings_);
|
||||
|
||||
void shutdown() override;
|
||||
~StorageMergeTree() override;
|
||||
|
@ -43,7 +43,7 @@ public:
|
||||
MergeTreeData::Mode mode_,
|
||||
const String & sign_column_, /// Для Collapsing режима.
|
||||
const Names & columns_to_sum_, /// Для Summing режима.
|
||||
const MergeTreeSettings & settings_ = MergeTreeSettings());
|
||||
const MergeTreeSettings & settings_);
|
||||
|
||||
void shutdown() override;
|
||||
~StorageReplicatedMergeTree() override;
|
||||
@ -124,6 +124,8 @@ public:
|
||||
UInt32 inserts_in_queue;
|
||||
UInt32 merges_in_queue;
|
||||
UInt32 queue_oldest_time;
|
||||
UInt32 inserts_oldest_time;
|
||||
UInt32 merges_oldest_time;
|
||||
UInt64 log_max_index;
|
||||
UInt64 log_pointer;
|
||||
UInt8 total_replicas;
|
||||
@ -282,7 +284,7 @@ private:
|
||||
MergeTreeData::Mode mode_,
|
||||
const String & sign_column_,
|
||||
const Names & columns_to_sum_,
|
||||
const MergeTreeSettings & settings_ = MergeTreeSettings());
|
||||
const MergeTreeSettings & settings_);
|
||||
|
||||
/// Инициализация.
|
||||
|
||||
|
@ -351,6 +351,26 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
else
|
||||
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")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
@ -632,7 +652,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
AggregateFunctionPtr nested = get(String(name.data(), name.size() - strlen("State")), argument_types, recursion_level + 1);
|
||||
return new AggregateFunctionState(nested);
|
||||
}
|
||||
else if (recursion_level == 0 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge")))
|
||||
else if (recursion_level <= 1 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge")))
|
||||
{
|
||||
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
|
||||
if (argument_types.size() != 1)
|
||||
@ -648,7 +668,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
|
||||
return new AggregateFunctionMerge(nested);
|
||||
}
|
||||
else if (recursion_level <= 1 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
|
||||
else if (recursion_level <= 2 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
|
||||
{
|
||||
if (argument_types.empty())
|
||||
throw Exception{
|
||||
@ -662,7 +682,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
AggregateFunctionPtr nested = get(String(name.data(), name.size() - 2), nested_dt, recursion_level + 1);
|
||||
return new AggregateFunctionIf(nested);
|
||||
}
|
||||
else if (recursion_level <= 2 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array")))
|
||||
else if (recursion_level <= 3 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array")))
|
||||
{
|
||||
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
|
||||
size_t num_agruments = argument_types.size();
|
||||
@ -675,7 +695,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[i]->getName() + " of argument #" + toString(i + 1) + " for aggregate function " + name + ". Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
AggregateFunctionPtr nested = get(String(name.data(), name.size() - strlen("Array")), nested_arguments, recursion_level + 2); /// + 2, чтобы ни один другой модификатор не мог идти перед Array
|
||||
AggregateFunctionPtr nested = get(String(name.data(), name.size() - strlen("Array")), nested_arguments, recursion_level + 3); /// + 3, чтобы ни один другой модификатор не мог идти перед Array
|
||||
return new AggregateFunctionArray(nested);
|
||||
}
|
||||
else
|
||||
@ -706,6 +726,7 @@ const AggregateFunctionFactory::FunctionNames & AggregateFunctionFactory::getFun
|
||||
"uniq",
|
||||
"uniqHLL12",
|
||||
"uniqExact",
|
||||
"uniqCombined",
|
||||
"uniqUpTo",
|
||||
"groupArray",
|
||||
"groupUniqArray",
|
||||
@ -744,14 +765,14 @@ bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int
|
||||
if (recursion_level <= 0 && name.size() > strlen("State") && !(strcmp(name.data() + name.size() - strlen("State"), "State")))
|
||||
return isAggregateFunctionName(String(name.data(), name.size() - strlen("State")), recursion_level + 1);
|
||||
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
|
||||
if (recursion_level <= 0 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge")))
|
||||
if (recursion_level <= 1 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge")))
|
||||
return isAggregateFunctionName(String(name.data(), name.size() - strlen("Merge")), recursion_level + 1);
|
||||
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
|
||||
if (recursion_level <= 1 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
|
||||
if (recursion_level <= 2 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
|
||||
return isAggregateFunctionName(String(name.data(), name.size() - 2), recursion_level + 1);
|
||||
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
|
||||
if (recursion_level <= 2 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array")))
|
||||
return isAggregateFunctionName(String(name.data(), name.size() - strlen("Array")), recursion_level + 2); /// + 2, чтобы ни один другой модификатор не мог идти перед Array
|
||||
if (recursion_level <= 3 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array")))
|
||||
return isAggregateFunctionName(String(name.data(), name.size() - strlen("Array")), recursion_level + 3); /// + 3, чтобы ни один другой модификатор не мог идти перед Array
|
||||
|
||||
return false;
|
||||
}
|
||||
|
@ -369,6 +369,19 @@ private:
|
||||
}
|
||||
|
||||
|
||||
/** Проверка для случая, когда в терминал вставляется многострочный запрос из буфера обмена.
|
||||
* Позволяет не начинать выполнение одной строчки запроса, пока весь запрос не будет вставлен.
|
||||
*/
|
||||
static bool hasDataInSTDIN()
|
||||
{
|
||||
timeval timeout = { 0, 0 };
|
||||
fd_set fds;
|
||||
FD_ZERO(&fds);
|
||||
FD_SET(STDIN_FILENO, &fds);
|
||||
return select(1, &fds, 0, 0, &timeout) == 1;
|
||||
}
|
||||
|
||||
|
||||
void loop()
|
||||
{
|
||||
String query;
|
||||
@ -395,7 +408,7 @@ private:
|
||||
|
||||
query += line;
|
||||
|
||||
if (!ends_with_backslash && (ends_with_semicolon || has_vertical_output_suffix || !config().has("multiline")))
|
||||
if (!ends_with_backslash && (ends_with_semicolon || has_vertical_output_suffix || (!config().has("multiline") && !hasDataInSTDIN())))
|
||||
{
|
||||
if (query != prev_query)
|
||||
{
|
||||
@ -464,6 +477,12 @@ private:
|
||||
copyData(in, out);
|
||||
}
|
||||
|
||||
process(line);
|
||||
}
|
||||
|
||||
|
||||
bool process(const String & line)
|
||||
{
|
||||
if (config().has("multiquery"))
|
||||
{
|
||||
/// Несколько запросов, разделенных ';'.
|
||||
@ -494,17 +513,20 @@ private:
|
||||
while (isWhitespace(*begin) || *begin == ';')
|
||||
++begin;
|
||||
|
||||
process(query, ast);
|
||||
if (!processSingleQuery(query, ast))
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
process(line);
|
||||
return processSingleQuery(line);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool process(const String & line, ASTPtr parsed_query_ = nullptr)
|
||||
bool processSingleQuery(const String & line, ASTPtr parsed_query_ = nullptr)
|
||||
{
|
||||
if (exit_strings.end() != exit_strings.find(line))
|
||||
return false;
|
||||
@ -838,15 +860,8 @@ private:
|
||||
}
|
||||
|
||||
|
||||
void onData(Block & block)
|
||||
void initBlockOutputStream(const Block & block)
|
||||
{
|
||||
if (written_progress_chars)
|
||||
clearProgress();
|
||||
|
||||
if (!block)
|
||||
return;
|
||||
|
||||
processed_rows += block.rows();
|
||||
if (!block_std_out)
|
||||
{
|
||||
String current_format = format;
|
||||
@ -869,8 +884,21 @@ private:
|
||||
block_std_out = context.getFormatFactory().getOutput(current_format, std_out, block);
|
||||
block_std_out->writePrefix();
|
||||
}
|
||||
}
|
||||
|
||||
/// Загаловочный блок с нулем строк использовался для инициализации block_std_out,
|
||||
|
||||
void onData(Block & block)
|
||||
{
|
||||
if (written_progress_chars)
|
||||
clearProgress();
|
||||
|
||||
if (!block)
|
||||
return;
|
||||
|
||||
processed_rows += block.rows();
|
||||
initBlockOutputStream(block);
|
||||
|
||||
/// Заголовочный блок с нулем строк использовался для инициализации block_std_out,
|
||||
/// выводить его не нужно
|
||||
if (block.rows() != 0)
|
||||
{
|
||||
@ -885,11 +913,13 @@ private:
|
||||
|
||||
void onTotals(Block & block)
|
||||
{
|
||||
initBlockOutputStream(block);
|
||||
block_std_out->setTotals(block);
|
||||
}
|
||||
|
||||
void onExtremes(Block & block)
|
||||
{
|
||||
initBlockOutputStream(block);
|
||||
block_std_out->setExtremes(block);
|
||||
}
|
||||
|
||||
|
@ -152,7 +152,7 @@ bool filterBlockWithQuery(ASTPtr query, Block & block, const Context & context)
|
||||
return false;
|
||||
|
||||
/// Распарсим и вычислим выражение.
|
||||
ExpressionAnalyzer analyzer(expression_ast, context, block.getColumnsList());
|
||||
ExpressionAnalyzer analyzer(expression_ast, context, {}, block.getColumnsList());
|
||||
ExpressionActionsPtr actions = analyzer.getActions(false);
|
||||
actions->execute(block);
|
||||
|
||||
|
@ -51,7 +51,7 @@ void Block::addDefaults(const NamesAndTypesList & required_columns)
|
||||
if (has(requested_column.name))
|
||||
continue;
|
||||
|
||||
ColumnWithNameAndType column_to_add;
|
||||
ColumnWithTypeAndName column_to_add;
|
||||
column_to_add.name = requested_column.name;
|
||||
column_to_add.type = requested_column.type;
|
||||
|
||||
@ -101,7 +101,7 @@ Block & Block::operator= (const Block & other)
|
||||
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())
|
||||
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);
|
||||
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))
|
||||
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())
|
||||
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())
|
||||
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);
|
||||
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);
|
||||
if (index_by_name.end() == it)
|
||||
@ -302,7 +302,13 @@ std::string Block::dumpStructure() const
|
||||
{
|
||||
if (it != data.begin())
|
||||
res << ", ";
|
||||
res << it->name << ' ' << it->type->getName() << ' ' << it->column->getName() << ' ' << it->column->size();
|
||||
|
||||
res << it->name << ' ' << it->type->getName();
|
||||
|
||||
if (it->column)
|
||||
res << ' ' << it->column->getName() << ' ' << it->column->size();
|
||||
else
|
||||
res << "nullptr";
|
||||
}
|
||||
return res.str();
|
||||
}
|
||||
@ -330,9 +336,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());
|
||||
}
|
||||
|
||||
|
||||
|
@ -37,7 +37,7 @@ Block AggregatingSortedBlockInputStream::readImpl()
|
||||
/// Заполним номера столбцов, которые нужно доагрегировать.
|
||||
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)
|
||||
|
@ -75,7 +75,7 @@ Block FilterBlockInputStream::readImpl()
|
||||
size_t filtered_rows = 0;
|
||||
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);
|
||||
filtered_rows = current_column.column->size();
|
||||
}
|
||||
@ -100,7 +100,7 @@ Block FilterBlockInputStream::readImpl()
|
||||
/// Фильтруем остальные столбцы.
|
||||
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))
|
||||
{
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <DB/DataStreams/BlockOutputStreamFromRowOutputStream.h>
|
||||
#include <DB/DataStreams/JSONRowOutputStream.h>
|
||||
#include <DB/DataStreams/JSONCompactRowOutputStream.h>
|
||||
#include <DB/DataStreams/TSKVRowOutputStream.h>
|
||||
#include <DB/DataStreams/PrettyCompactMonoBlockOutputStream.h>
|
||||
#include <DB/DataStreams/FormatFactory.h>
|
||||
|
||||
@ -25,7 +26,7 @@ namespace DB
|
||||
{
|
||||
|
||||
BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & buf,
|
||||
Block & sample, size_t max_block_size) const
|
||||
const Block & sample, size_t max_block_size) const
|
||||
{
|
||||
if (name == "Native")
|
||||
return new NativeBlockInputStream(buf);
|
||||
@ -47,7 +48,7 @@ BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & bu
|
||||
|
||||
|
||||
BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & buf,
|
||||
Block & sample) const
|
||||
const Block & sample) const
|
||||
{
|
||||
if (name == "Native")
|
||||
return new NativeBlockOutputStream(buf);
|
||||
@ -83,6 +84,8 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer &
|
||||
return new BlockOutputStreamFromRowOutputStream(new JSONRowOutputStream(buf, sample));
|
||||
else if (name == "JSONCompact")
|
||||
return new BlockOutputStreamFromRowOutputStream(new JSONCompactRowOutputStream(buf, sample));
|
||||
else if (name == "TSKV")
|
||||
return new BlockOutputStreamFromRowOutputStream(new TSKVRowOutputStream(buf, sample));
|
||||
else if (name == "Null")
|
||||
return new NullBlockOutputStream;
|
||||
else if (name == "PrettyCompactMonoBlock")
|
||||
|
@ -58,7 +58,7 @@ void JSONCompactRowOutputStream::writeTotals()
|
||||
if (i != 0)
|
||||
writeChar(',', *ostr);
|
||||
|
||||
const ColumnWithNameAndType & column = totals.getByPosition(i);
|
||||
const ColumnWithTypeAndName & column = totals.getByPosition(i);
|
||||
column.type->serializeTextJSON((*column.column)[0], *ostr);
|
||||
}
|
||||
|
||||
@ -79,7 +79,7 @@ static void writeExtremesElement(const char * title, const Block & extremes, siz
|
||||
if (i != 0)
|
||||
writeChar(',', ostr);
|
||||
|
||||
const ColumnWithNameAndType & column = extremes.getByPosition(i);
|
||||
const ColumnWithTypeAndName & column = extremes.getByPosition(i);
|
||||
column.type->serializeTextJSON((*column.column)[row_num], ostr);
|
||||
}
|
||||
|
||||
|
@ -139,7 +139,7 @@ void JSONRowOutputStream::writeTotals()
|
||||
size_t totals_columns = totals.columns();
|
||||
for (size_t i = 0; i < totals_columns; ++i)
|
||||
{
|
||||
const ColumnWithNameAndType & column = totals.getByPosition(i);
|
||||
const ColumnWithTypeAndName & column = totals.getByPosition(i);
|
||||
|
||||
if (i != 0)
|
||||
writeCString(",\n", *ostr);
|
||||
@ -166,7 +166,7 @@ static void writeExtremesElement(const char * title, const Block & extremes, siz
|
||||
size_t extremes_columns = extremes.columns();
|
||||
for (size_t i = 0; i < extremes_columns; ++i)
|
||||
{
|
||||
const ColumnWithNameAndType & column = extremes.getByPosition(i);
|
||||
const ColumnWithTypeAndName & column = extremes.getByPosition(i);
|
||||
|
||||
if (i != 0)
|
||||
writeCString(",\n", ostr);
|
||||
|
@ -62,7 +62,7 @@ Block NativeBlockInputStream::readImpl()
|
||||
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
ColumnWithNameAndType column;
|
||||
ColumnWithTypeAndName column;
|
||||
|
||||
/// Имя
|
||||
readStringBinary(column.name, istr);
|
||||
|
@ -76,7 +76,7 @@ void NativeBlockOutputStream::write(const Block & block)
|
||||
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
const ColumnWithNameAndType & column = block.getByPosition(i);
|
||||
const ColumnWithTypeAndName & column = block.getByPosition(i);
|
||||
|
||||
/// Имя
|
||||
writeStringBinary(column.name, ostr);
|
||||
|
@ -32,7 +32,7 @@ void PrettyBlockOutputStream::calculateWidths(Block & block, Widths_t & max_widt
|
||||
/// Вычислим ширину всех значений
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
ColumnWithNameAndType column;
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = visible_width_type;
|
||||
column.name = "visibleWidth(" + block.getByPosition(i).name + ")";
|
||||
|
||||
@ -137,7 +137,7 @@ void PrettyBlockOutputStream::write(const Block & block_)
|
||||
if (i != 0)
|
||||
writeCString(" ┃ ", ostr);
|
||||
|
||||
const ColumnWithNameAndType & col = block.getByPosition(i);
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(i);
|
||||
|
||||
if (!no_escapes)
|
||||
writeCString("\033[1m", ostr);
|
||||
@ -176,7 +176,7 @@ void PrettyBlockOutputStream::write(const Block & block_)
|
||||
if (j != 0)
|
||||
writeCString(" │ ", ostr);
|
||||
|
||||
const ColumnWithNameAndType & col = block.getByPosition(j);
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(j);
|
||||
|
||||
if (col.type->isNumeric())
|
||||
{
|
||||
|
@ -21,7 +21,7 @@ void PrettyCompactBlockOutputStream::writeHeader(
|
||||
if (i != 0)
|
||||
writeCString("─┬─", ostr);
|
||||
|
||||
const ColumnWithNameAndType & col = block.getByPosition(i);
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(i);
|
||||
|
||||
if (col.type->isNumeric())
|
||||
{
|
||||
@ -83,7 +83,7 @@ void PrettyCompactBlockOutputStream::writeRow(
|
||||
if (j != 0)
|
||||
writeCString(" │ ", ostr);
|
||||
|
||||
const ColumnWithNameAndType & col = block.getByPosition(j);
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(j);
|
||||
|
||||
if (col.type->isNumeric())
|
||||
{
|
||||
|
@ -40,7 +40,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
|
||||
if (i != 0)
|
||||
writeCString(" ", ostr);
|
||||
|
||||
const ColumnWithNameAndType & col = block.getByPosition(i);
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(i);
|
||||
|
||||
if (col.type->isNumeric())
|
||||
{
|
||||
@ -74,7 +74,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
|
||||
if (j != 0)
|
||||
writeCString(" ", ostr);
|
||||
|
||||
const ColumnWithNameAndType & col = block.getByPosition(j);
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(j);
|
||||
|
||||
if (col.type->isNumeric())
|
||||
{
|
||||
|
@ -63,7 +63,7 @@ Block SummingSortedBlockInputStream::readImpl()
|
||||
*/
|
||||
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
|
||||
if (const auto array_type = typeid_cast<const DataTypeArray *>(column.type.get()))
|
||||
|
37
dbms/src/DataStreams/TSKVRowOutputStream.cpp
Normal file
37
dbms/src/DataStreams/TSKVRowOutputStream.cpp
Normal 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;
|
||||
}
|
||||
|
||||
}
|
@ -9,7 +9,7 @@ void TabSeparatedBlockOutputStream::write(const Block & block)
|
||||
size_t columns = block.columns();
|
||||
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();
|
||||
for (size_t j = 0; j < rows; ++j)
|
||||
|
@ -9,7 +9,7 @@ static void finalize(Block & block)
|
||||
{
|
||||
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);
|
||||
if (unfinalized_column)
|
||||
{
|
||||
@ -106,7 +106,7 @@ Block TotalsHavingBlockInputStream::readImpl()
|
||||
|
||||
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);
|
||||
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)
|
||||
{
|
||||
const ColumnWithNameAndType & current = block.getByPosition(i);
|
||||
const ColumnWithTypeAndName & current = block.getByPosition(i);
|
||||
const ColumnAggregateFunction * column = typeid_cast<const ColumnAggregateFunction *>(&*current.column);
|
||||
|
||||
if (!column)
|
||||
@ -143,7 +143,7 @@ void TotalsHavingBlockInputStream::addToTotals(Block & totals, Block & block, co
|
||||
{
|
||||
ColumnPtr new_column = current.type->createColumn();
|
||||
new_column->insertDefault();
|
||||
totals.insert(ColumnWithNameAndType(new_column, current.type, current.name));
|
||||
totals.insert(ColumnWithTypeAndName(new_column, current.type, current.name));
|
||||
}
|
||||
continue;
|
||||
}
|
||||
@ -156,7 +156,7 @@ void TotalsHavingBlockInputStream::addToTotals(Block & totals, Block & block, co
|
||||
{
|
||||
function = column->getAggregateFunction();
|
||||
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());
|
||||
function->create(data);
|
||||
|
@ -31,7 +31,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
Block block;
|
||||
|
||||
ColumnWithNameAndType column_x;
|
||||
ColumnWithTypeAndName column_x;
|
||||
column_x.name = "x";
|
||||
column_x.type = new DataTypeInt16;
|
||||
ColumnInt16 * x = new ColumnInt16;
|
||||
@ -46,7 +46,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
const char * strings[] = {"abc", "def", "abcd", "defg", "ac"};
|
||||
|
||||
ColumnWithNameAndType column_s1;
|
||||
ColumnWithTypeAndName column_s1;
|
||||
column_s1.name = "s1";
|
||||
column_s1.type = new DataTypeString;
|
||||
column_s1.column = new ColumnString;
|
||||
@ -56,7 +56,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
block.insert(column_s1);
|
||||
|
||||
ColumnWithNameAndType column_s2;
|
||||
ColumnWithTypeAndName column_s2;
|
||||
column_s2.name = "s2";
|
||||
column_s2.type = new DataTypeString;
|
||||
column_s2.column = new ColumnString;
|
||||
@ -86,7 +86,7 @@ int main(int argc, char ** argv)
|
||||
Block sample;
|
||||
for (DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it)
|
||||
{
|
||||
ColumnWithNameAndType col;
|
||||
ColumnWithTypeAndName col;
|
||||
col.type = *it;
|
||||
sample.insert(col);
|
||||
}
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/Core/ColumnWithNameAndType.h>
|
||||
#include <DB/Core/ColumnWithTypeAndName.h>
|
||||
|
||||
#include <DB/IO/ReadBufferFromIStream.h>
|
||||
#include <DB/IO/WriteBufferFromOStream.h>
|
||||
@ -28,13 +28,13 @@ int main(int argc, char ** argv)
|
||||
{
|
||||
DB::Block sample;
|
||||
|
||||
DB::ColumnWithNameAndType col1;
|
||||
DB::ColumnWithTypeAndName col1;
|
||||
col1.name = "col1";
|
||||
col1.type = new DB::DataTypeUInt64;
|
||||
col1.column = col1.type->createColumn();
|
||||
sample.insert(col1);
|
||||
|
||||
DB::ColumnWithNameAndType col2;
|
||||
DB::ColumnWithTypeAndName col2;
|
||||
col2.name = "col2";
|
||||
col2.type = new DB::DataTypeString;
|
||||
col2.column = col2.type->createColumn();
|
||||
|
@ -98,7 +98,7 @@ int main(int argc, char ** argv)
|
||||
Block sample;
|
||||
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.type = it->type;
|
||||
elem.column = elem.type->createColumn();
|
||||
|
@ -32,7 +32,7 @@ int main(int argc, char ** argv)
|
||||
Block block1;
|
||||
|
||||
{
|
||||
ColumnWithNameAndType column1;
|
||||
ColumnWithTypeAndName column1;
|
||||
column1.name = "Sign";
|
||||
column1.type = new DataTypeInt8;
|
||||
column1.column = new ColumnInt8;
|
||||
@ -40,7 +40,7 @@ int main(int argc, char ** argv)
|
||||
column1.column->insert(DB::Int64(-1));
|
||||
block1.insert(column1);
|
||||
|
||||
ColumnWithNameAndType column2;
|
||||
ColumnWithTypeAndName column2;
|
||||
column2.name = "CounterID";
|
||||
column2.type = new DataTypeUInt32;
|
||||
column2.column = new ColumnUInt32;
|
||||
@ -52,7 +52,7 @@ int main(int argc, char ** argv)
|
||||
Block block2;
|
||||
|
||||
{
|
||||
ColumnWithNameAndType column1;
|
||||
ColumnWithTypeAndName column1;
|
||||
column1.name = "Sign";
|
||||
column1.type = new DataTypeInt8;
|
||||
column1.column = new ColumnInt8;
|
||||
@ -60,7 +60,7 @@ int main(int argc, char ** argv)
|
||||
column1.column->insert(DB::Int64(1));
|
||||
block2.insert(column1);
|
||||
|
||||
ColumnWithNameAndType column2;
|
||||
ColumnWithTypeAndName column2;
|
||||
column2.name = "CounterID";
|
||||
column2.type = new DataTypeUInt32;
|
||||
column2.column = new ColumnUInt32;
|
||||
|
@ -38,9 +38,8 @@ int main(int argc, char ** argv)
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
|
||||
|
||||
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;
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendProjectResult(chain, false);
|
||||
|
@ -44,9 +44,8 @@ int main(int argc, char ** argv)
|
||||
std::cerr << ast->getTreeID() << std::endl;
|
||||
|
||||
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;
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendProjectResult(chain, false);
|
||||
|
@ -98,8 +98,6 @@ int main(int argc, char ** argv)
|
||||
|
||||
Context context;
|
||||
|
||||
context.getColumns() = *names_and_types_list;
|
||||
|
||||
std::string input = "SELECT UniqID, URL, CounterID, IsLink WHERE URL = 'http://mail.yandex.ru/neo2/#inbox'";
|
||||
ParserSelectQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
|
||||
@ -114,7 +112,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
/// читаем из неё, применяем выражение, фильтруем, и пишем в tsv виде в консоль
|
||||
|
||||
ExpressionAnalyzer analyzer(ast, context, context.getColumns());
|
||||
ExpressionAnalyzer analyzer(ast, context, nullptr, *names_and_types_list);
|
||||
ExpressionActionsChain chain;
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendWhere(chain, false);
|
||||
|
@ -62,9 +62,8 @@ int main(int argc, char ** argv)
|
||||
std::cerr << std::endl;
|
||||
|
||||
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;
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendProjectResult(chain, false);
|
||||
|
@ -51,7 +51,7 @@ int main(int argc, char ** argv)
|
||||
Block sample;
|
||||
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.type = it->type;
|
||||
elem.column = elem.type->createColumn();
|
||||
|
@ -136,7 +136,7 @@ int main(int argc, char ** argv)
|
||||
Block sample;
|
||||
for (DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it)
|
||||
{
|
||||
ColumnWithNameAndType col;
|
||||
ColumnWithTypeAndName col;
|
||||
col.type = *it;
|
||||
sample.insert(col);
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user