mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 18:12:02 +00:00
Merge branch 'master' into master
This commit is contained in:
commit
edc2dc45b6
@ -13,6 +13,7 @@
|
|||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
#include <Poco/String.h>
|
#include <Poco/String.h>
|
||||||
|
#include <DataTypes/DataTypeWithDictionary.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -41,6 +42,20 @@ void AggregateFunctionFactory::registerFunction(const String & name, Creator cre
|
|||||||
ErrorCodes::LOGICAL_ERROR);
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static DataTypes convertTypesWithDictionaryToNested(const DataTypes & types)
|
||||||
|
{
|
||||||
|
DataTypes res_types;
|
||||||
|
res_types.reserve(types.size());
|
||||||
|
for (const auto & type : types)
|
||||||
|
{
|
||||||
|
if (auto * type_with_dict = typeid_cast<const DataTypeWithDictionary *>(type.get()))
|
||||||
|
res_types.push_back(type_with_dict->getDictionaryType());
|
||||||
|
else
|
||||||
|
res_types.push_back(type);
|
||||||
|
}
|
||||||
|
|
||||||
|
return res_types;
|
||||||
|
}
|
||||||
|
|
||||||
AggregateFunctionPtr AggregateFunctionFactory::get(
|
AggregateFunctionPtr AggregateFunctionFactory::get(
|
||||||
const String & name,
|
const String & name,
|
||||||
@ -48,6 +63,8 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
|||||||
const Array & parameters,
|
const Array & parameters,
|
||||||
int recursion_level) const
|
int recursion_level) const
|
||||||
{
|
{
|
||||||
|
auto type_without_dictionary = convertTypesWithDictionaryToNested(argument_types);
|
||||||
|
|
||||||
/// If one of types is Nullable, we apply aggregate function combinator "Null".
|
/// If one of types is Nullable, we apply aggregate function combinator "Null".
|
||||||
|
|
||||||
if (std::any_of(argument_types.begin(), argument_types.end(),
|
if (std::any_of(argument_types.begin(), argument_types.end(),
|
||||||
@ -57,7 +74,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
|||||||
if (!combinator)
|
if (!combinator)
|
||||||
throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
DataTypes nested_types = combinator->transformArguments(argument_types);
|
DataTypes nested_types = combinator->transformArguments(type_without_dictionary);
|
||||||
|
|
||||||
AggregateFunctionPtr nested_function;
|
AggregateFunctionPtr nested_function;
|
||||||
|
|
||||||
@ -70,7 +87,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
|||||||
return combinator->transformAggregateFunction(nested_function, argument_types, parameters);
|
return combinator->transformAggregateFunction(nested_function, argument_types, parameters);
|
||||||
}
|
}
|
||||||
|
|
||||||
auto res = getImpl(name, argument_types, parameters, recursion_level);
|
auto res = getImpl(name, type_without_dictionary, parameters, recursion_level);
|
||||||
if (!res)
|
if (!res)
|
||||||
throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR);
|
||||||
return res;
|
return res;
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <IO/WriteBufferFromArena.h>
|
#include <IO/WriteBufferFromArena.h>
|
||||||
#include <Common/SipHash.h>
|
#include <Common/SipHash.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Columns/ColumnsCommon.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -161,6 +162,25 @@ ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limi
|
|||||||
return std::move(res);
|
return std::move(res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ColumnPtr ColumnAggregateFunction::index(const IColumn & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
return selectIndexImpl(*this, indexes, limit);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename Type>
|
||||||
|
ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
auto res = createView();
|
||||||
|
|
||||||
|
res->getData().resize(limit);
|
||||||
|
for (size_t i = 0; i < limit; ++i)
|
||||||
|
res->getData()[i] = getData()[indexes[i]];
|
||||||
|
|
||||||
|
return std::move(res);
|
||||||
|
}
|
||||||
|
|
||||||
|
INSTANTIATE_INDEX_IMPL(ColumnAggregateFunction);
|
||||||
|
|
||||||
/// Is required to support operations with Set
|
/// Is required to support operations with Set
|
||||||
void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) const
|
void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) const
|
||||||
{
|
{
|
||||||
|
@ -156,6 +156,11 @@ public:
|
|||||||
|
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
|
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
||||||
|
|
||||||
|
template <typename Type>
|
||||||
|
ColumnPtr indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const;
|
||||||
|
|
||||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||||
|
|
||||||
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override;
|
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override;
|
||||||
|
@ -626,6 +626,44 @@ ColumnPtr ColumnArray::permute(const Permutation & perm, size_t limit) const
|
|||||||
return std::move(res);
|
return std::move(res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ColumnPtr ColumnArray::index(const IColumn & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
return selectIndexImpl(*this, indexes, limit);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
ColumnPtr ColumnArray::indexImpl(const PaddedPODArray<T> & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
if (limit == 0)
|
||||||
|
return ColumnArray::create(data);
|
||||||
|
|
||||||
|
/// Convert indexes to UInt64 in case of overflow.
|
||||||
|
auto nested_indexes_column = ColumnUInt64::create();
|
||||||
|
PaddedPODArray<UInt64> & nested_indexes = nested_indexes_column->getData();
|
||||||
|
nested_indexes.reserve(getOffsets().back());
|
||||||
|
|
||||||
|
auto res = ColumnArray::create(data->cloneEmpty());
|
||||||
|
|
||||||
|
Offsets & res_offsets = res->getOffsets();
|
||||||
|
res_offsets.resize(limit);
|
||||||
|
size_t current_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < limit; ++i)
|
||||||
|
{
|
||||||
|
for (size_t j = 0; j < sizeAt(indexes[i]); ++j)
|
||||||
|
nested_indexes.push_back(offsetAt(indexes[i]) + j);
|
||||||
|
current_offset += sizeAt(indexes[i]);
|
||||||
|
res_offsets[i] = current_offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (current_offset != 0)
|
||||||
|
res->data = data->index(*nested_indexes_column, current_offset);
|
||||||
|
|
||||||
|
return std::move(res);
|
||||||
|
}
|
||||||
|
|
||||||
|
INSTANTIATE_INDEX_IMPL(ColumnArray);
|
||||||
|
|
||||||
void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
|
void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
|
||||||
{
|
{
|
||||||
size_t s = size();
|
size_t s = size();
|
||||||
|
@ -71,6 +71,8 @@ public:
|
|||||||
void popBack(size_t n) override;
|
void popBack(size_t n) override;
|
||||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
||||||
|
template <typename Type> ColumnPtr indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const;
|
||||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override;
|
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override;
|
||||||
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
|
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
|
||||||
void reserve(size_t n) override;
|
void reserve(size_t n) override;
|
||||||
|
@ -63,6 +63,18 @@ ColumnPtr ColumnConst::permute(const Permutation & perm, size_t limit) const
|
|||||||
return ColumnConst::create(data, limit);
|
return ColumnConst::create(data, limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ColumnPtr ColumnConst::index(const IColumn & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
if (limit == 0)
|
||||||
|
limit = indexes.size();
|
||||||
|
|
||||||
|
if (indexes.size() < limit)
|
||||||
|
throw Exception("Size of indexes (" + toString(indexes.size()) + ") is less than required (" + toString(limit) + ")",
|
||||||
|
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
return ColumnConst::create(data, limit);
|
||||||
|
}
|
||||||
|
|
||||||
MutableColumns ColumnConst::scatter(ColumnIndex num_columns, const Selector & selector) const
|
MutableColumns ColumnConst::scatter(ColumnIndex num_columns, const Selector & selector) const
|
||||||
{
|
{
|
||||||
if (s != selector.size())
|
if (s != selector.size())
|
||||||
|
@ -153,6 +153,7 @@ public:
|
|||||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
||||||
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
|
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
|
||||||
|
|
||||||
size_t byteSize() const override
|
size_t byteSize() const override
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Columns/ColumnFixedString.h>
|
#include <Columns/ColumnFixedString.h>
|
||||||
|
#include <Columns/ColumnsCommon.h>
|
||||||
|
|
||||||
#include <Common/Arena.h>
|
#include <Common/Arena.h>
|
||||||
#include <Common/SipHash.h>
|
#include <Common/SipHash.h>
|
||||||
@ -258,6 +259,32 @@ ColumnPtr ColumnFixedString::permute(const Permutation & perm, size_t limit) con
|
|||||||
return std::move(res);
|
return std::move(res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnPtr ColumnFixedString::index(const IColumn & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
return selectIndexImpl(*this, indexes, limit);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <typename Type>
|
||||||
|
ColumnPtr ColumnFixedString::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
if (limit == 0)
|
||||||
|
return ColumnFixedString::create(n);
|
||||||
|
|
||||||
|
auto res = ColumnFixedString::create(n);
|
||||||
|
|
||||||
|
Chars_t & res_chars = res->chars;
|
||||||
|
|
||||||
|
res_chars.resize(n * limit);
|
||||||
|
|
||||||
|
size_t offset = 0;
|
||||||
|
for (size_t i = 0; i < limit; ++i, offset += n)
|
||||||
|
memcpySmallAllowReadWriteOverflow15(&res_chars[offset], &chars[indexes[i] * n], n);
|
||||||
|
|
||||||
|
return std::move(res);
|
||||||
|
}
|
||||||
|
|
||||||
ColumnPtr ColumnFixedString::replicate(const Offsets & offsets) const
|
ColumnPtr ColumnFixedString::replicate(const Offsets & offsets) const
|
||||||
{
|
{
|
||||||
size_t col_size = size();
|
size_t col_size = size();
|
||||||
|
@ -108,6 +108,11 @@ public:
|
|||||||
|
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
|
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
||||||
|
|
||||||
|
template <typename Type>
|
||||||
|
ColumnPtr indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const;
|
||||||
|
|
||||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||||
|
|
||||||
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override
|
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override
|
||||||
|
@ -88,6 +88,15 @@ ColumnPtr ColumnFunction::permute(const Permutation & perm, size_t limit) const
|
|||||||
return ColumnFunction::create(limit, function, capture);
|
return ColumnFunction::create(limit, function, capture);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ColumnPtr ColumnFunction::index(const IColumn & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
ColumnsWithTypeAndName capture = captured_columns;
|
||||||
|
for (auto & column : capture)
|
||||||
|
column.column = column.column->index(indexes, limit);
|
||||||
|
|
||||||
|
return ColumnFunction::create(limit, function, capture);
|
||||||
|
}
|
||||||
|
|
||||||
std::vector<MutableColumnPtr> ColumnFunction::scatter(IColumn::ColumnIndex num_columns,
|
std::vector<MutableColumnPtr> ColumnFunction::scatter(IColumn::ColumnIndex num_columns,
|
||||||
const IColumn::Selector & selector) const
|
const IColumn::Selector & selector) const
|
||||||
{
|
{
|
||||||
|
@ -33,6 +33,7 @@ public:
|
|||||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
||||||
void insertDefault() override;
|
void insertDefault() override;
|
||||||
void popBack(size_t n) override;
|
void popBack(size_t n) override;
|
||||||
std::vector<MutableColumnPtr> scatter(IColumn::ColumnIndex num_columns,
|
std::vector<MutableColumnPtr> scatter(IColumn::ColumnIndex num_columns,
|
||||||
|
@ -166,6 +166,13 @@ ColumnPtr ColumnNullable::permute(const Permutation & perm, size_t limit) const
|
|||||||
return ColumnNullable::create(permuted_data, permuted_null_map);
|
return ColumnNullable::create(permuted_data, permuted_null_map);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ColumnPtr ColumnNullable::index(const IColumn & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
ColumnPtr indexed_data = getNestedColumn().index(indexes, limit);
|
||||||
|
ColumnPtr indexed_null_map = getNullMapColumn().index(indexes, limit);
|
||||||
|
return ColumnNullable::create(indexed_data, indexed_null_map);
|
||||||
|
}
|
||||||
|
|
||||||
int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const
|
int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const
|
||||||
{
|
{
|
||||||
/// NULL values share the properties of NaN values.
|
/// NULL values share the properties of NaN values.
|
||||||
|
@ -65,6 +65,7 @@ public:
|
|||||||
void popBack(size_t n) override;
|
void popBack(size_t n) override;
|
||||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
||||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override;
|
int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override;
|
||||||
void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override;
|
void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override;
|
||||||
void reserve(size_t n) override;
|
void reserve(size_t n) override;
|
||||||
|
@ -159,6 +159,48 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnPtr ColumnString::index(const IColumn & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
return selectIndexImpl(*this, indexes, limit);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename Type>
|
||||||
|
ColumnPtr ColumnString::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
if (limit == 0)
|
||||||
|
return ColumnString::create();
|
||||||
|
|
||||||
|
auto res = ColumnString::create();
|
||||||
|
|
||||||
|
Chars_t & res_chars = res->chars;
|
||||||
|
Offsets & res_offsets = res->offsets;
|
||||||
|
|
||||||
|
|
||||||
|
size_t new_chars_size = 0;
|
||||||
|
for (size_t i = 0; i < limit; ++i)
|
||||||
|
new_chars_size += sizeAt(indexes[i]);
|
||||||
|
res_chars.resize(new_chars_size);
|
||||||
|
|
||||||
|
res_offsets.resize(limit);
|
||||||
|
|
||||||
|
Offset current_new_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < limit; ++i)
|
||||||
|
{
|
||||||
|
size_t j = indexes[i];
|
||||||
|
size_t string_offset = j == 0 ? 0 : offsets[j - 1];
|
||||||
|
size_t string_size = offsets[j] - string_offset;
|
||||||
|
|
||||||
|
memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size);
|
||||||
|
|
||||||
|
current_new_offset += string_size;
|
||||||
|
res_offsets[i] = current_new_offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
return std::move(res);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
template <bool positive>
|
template <bool positive>
|
||||||
struct ColumnString::less
|
struct ColumnString::less
|
||||||
{
|
{
|
||||||
|
@ -220,6 +220,11 @@ public:
|
|||||||
|
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
|
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
||||||
|
|
||||||
|
template <typename Type>
|
||||||
|
ColumnPtr indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const;
|
||||||
|
|
||||||
void insertDefault() override
|
void insertDefault() override
|
||||||
{
|
{
|
||||||
chars.push_back(0);
|
chars.push_back(0);
|
||||||
|
@ -181,6 +181,17 @@ ColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) const
|
|||||||
return ColumnTuple::create(new_columns);
|
return ColumnTuple::create(new_columns);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ColumnPtr ColumnTuple::index(const IColumn & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
const size_t tuple_size = columns.size();
|
||||||
|
Columns new_columns(tuple_size);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < tuple_size; ++i)
|
||||||
|
new_columns[i] = columns[i]->index(indexes, limit);
|
||||||
|
|
||||||
|
return ColumnTuple::create(new_columns);
|
||||||
|
}
|
||||||
|
|
||||||
ColumnPtr ColumnTuple::replicate(const Offsets & offsets) const
|
ColumnPtr ColumnTuple::replicate(const Offsets & offsets) const
|
||||||
{
|
{
|
||||||
const size_t tuple_size = columns.size();
|
const size_t tuple_size = columns.size();
|
||||||
|
@ -60,6 +60,7 @@ public:
|
|||||||
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
||||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
||||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||||
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override;
|
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override;
|
||||||
void gather(ColumnGathererStream & gatherer_stream) override;
|
void gather(ColumnGathererStream & gatherer_stream) override;
|
||||||
|
512
dbms/src/Columns/ColumnUnique.h
Normal file
512
dbms/src/Columns/ColumnUnique.h
Normal file
@ -0,0 +1,512 @@
|
|||||||
|
#pragma once
|
||||||
|
#include <Columns/IColumnUnique.h>
|
||||||
|
#include <Columns/ReverseIndex.h>
|
||||||
|
|
||||||
|
#include <Columns/ColumnVector.h>
|
||||||
|
#include <Columns/ColumnNullable.h>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnFixedString.h>
|
||||||
|
|
||||||
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
#include <DataTypes/NumberTraits.h>
|
||||||
|
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <ext/range.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
class ColumnUnique final : public COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType>>
|
||||||
|
{
|
||||||
|
friend class COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType>>;
|
||||||
|
|
||||||
|
private:
|
||||||
|
explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable);
|
||||||
|
explicit ColumnUnique(const IDataType & type);
|
||||||
|
ColumnUnique(const ColumnUnique & other);
|
||||||
|
|
||||||
|
public:
|
||||||
|
MutableColumnPtr cloneEmpty() const override;
|
||||||
|
|
||||||
|
const ColumnPtr & getNestedColumn() const override;
|
||||||
|
const ColumnPtr & getNestedNotNullableColumn() const override { return column_holder; }
|
||||||
|
|
||||||
|
size_t uniqueInsert(const Field & x) override;
|
||||||
|
size_t uniqueInsertFrom(const IColumn & src, size_t n) override;
|
||||||
|
MutableColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
||||||
|
IColumnUnique::IndexesWithOverflow uniqueInsertRangeWithOverflow(const IColumn & src, size_t start, size_t length,
|
||||||
|
size_t max_dictionary_size) override;
|
||||||
|
size_t uniqueInsertData(const char * pos, size_t length) override;
|
||||||
|
size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) override;
|
||||||
|
size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) override;
|
||||||
|
|
||||||
|
size_t getDefaultValueIndex() const override { return is_nullable ? 1 : 0; }
|
||||||
|
size_t getNullValueIndex() const override;
|
||||||
|
bool canContainNulls() const override { return is_nullable; }
|
||||||
|
|
||||||
|
Field operator[](size_t n) const override { return (*getNestedColumn())[n]; }
|
||||||
|
void get(size_t n, Field & res) const override { getNestedColumn()->get(n, res); }
|
||||||
|
StringRef getDataAt(size_t n) const override { return getNestedColumn()->getDataAt(n); }
|
||||||
|
StringRef getDataAtWithTerminatingZero(size_t n) const override
|
||||||
|
{
|
||||||
|
return getNestedColumn()->getDataAtWithTerminatingZero(n);
|
||||||
|
}
|
||||||
|
UInt64 get64(size_t n) const override { return getNestedColumn()->get64(n); }
|
||||||
|
UInt64 getUInt(size_t n) const override { return getNestedColumn()->getUInt(n); }
|
||||||
|
Int64 getInt(size_t n) const override { return getNestedColumn()->getInt(n); }
|
||||||
|
bool isNullAt(size_t n) const override { return is_nullable && n == getNullValueIndex(); }
|
||||||
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
||||||
|
{
|
||||||
|
return column_holder->serializeValueIntoArena(n, arena, begin);
|
||||||
|
}
|
||||||
|
void updateHashWithValue(size_t n, SipHash & hash) const override
|
||||||
|
{
|
||||||
|
return getNestedColumn()->updateHashWithValue(n, hash);
|
||||||
|
}
|
||||||
|
|
||||||
|
int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override
|
||||||
|
{
|
||||||
|
auto & column_unique = static_cast<const IColumnUnique&>(rhs);
|
||||||
|
return getNestedColumn()->compareAt(n, m, *column_unique.getNestedColumn(), nan_direction_hint);
|
||||||
|
}
|
||||||
|
|
||||||
|
void getExtremes(Field & min, Field & max) const override { column_holder->getExtremes(min, max); }
|
||||||
|
bool valuesHaveFixedSize() const override { return column_holder->valuesHaveFixedSize(); }
|
||||||
|
bool isFixedAndContiguous() const override { return column_holder->isFixedAndContiguous(); }
|
||||||
|
size_t sizeOfValueIfFixed() const override { return column_holder->sizeOfValueIfFixed(); }
|
||||||
|
bool isNumeric() const override { return column_holder->isNumeric(); }
|
||||||
|
|
||||||
|
size_t byteSize() const override { return column_holder->byteSize(); }
|
||||||
|
size_t allocatedBytes() const override
|
||||||
|
{
|
||||||
|
return column_holder->allocatedBytes()
|
||||||
|
+ index.allocatedBytes()
|
||||||
|
+ (cached_null_mask ? cached_null_mask->allocatedBytes() : 0);
|
||||||
|
}
|
||||||
|
void forEachSubcolumn(IColumn::ColumnCallback callback) override
|
||||||
|
{
|
||||||
|
callback(column_holder);
|
||||||
|
index.setColumn(getRawColumnPtr());
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
|
||||||
|
ColumnPtr column_holder;
|
||||||
|
bool is_nullable;
|
||||||
|
ReverseIndex<UInt64, ColumnType> index;
|
||||||
|
|
||||||
|
/// For DataTypeNullable, stores null map.
|
||||||
|
mutable ColumnPtr cached_null_mask;
|
||||||
|
mutable ColumnPtr cached_column_nullable;
|
||||||
|
|
||||||
|
static size_t numSpecialValues(bool is_nullable) { return is_nullable ? 2 : 1; }
|
||||||
|
size_t numSpecialValues() const { return numSpecialValues(is_nullable); }
|
||||||
|
|
||||||
|
ColumnType * getRawColumnPtr() { return static_cast<ColumnType *>(column_holder->assumeMutable().get()); }
|
||||||
|
const ColumnType * getRawColumnPtr() const { return static_cast<const ColumnType *>(column_holder.get()); }
|
||||||
|
|
||||||
|
template <typename IndexType>
|
||||||
|
MutableColumnPtr uniqueInsertRangeImpl(
|
||||||
|
const IColumn & src,
|
||||||
|
size_t start,
|
||||||
|
size_t length,
|
||||||
|
size_t num_added_rows,
|
||||||
|
typename ColumnVector<IndexType>::MutablePtr && positions_column,
|
||||||
|
ReverseIndex<UInt64, ColumnType> * secondary_index,
|
||||||
|
size_t max_dictionary_size);
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
MutableColumnPtr ColumnUnique<ColumnType>::cloneEmpty() const
|
||||||
|
{
|
||||||
|
return ColumnUnique<ColumnType>::create(column_holder->cloneResized(numSpecialValues()), is_nullable);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
ColumnUnique<ColumnType>::ColumnUnique(const ColumnUnique & other)
|
||||||
|
: column_holder(other.column_holder)
|
||||||
|
, is_nullable(other.is_nullable)
|
||||||
|
, index(numSpecialValues(is_nullable), 0)
|
||||||
|
{
|
||||||
|
index.setColumn(getRawColumnPtr());
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
ColumnUnique<ColumnType>::ColumnUnique(const IDataType & type)
|
||||||
|
: is_nullable(type.isNullable())
|
||||||
|
, index(numSpecialValues(is_nullable), 0)
|
||||||
|
{
|
||||||
|
const auto & holder_type = is_nullable ? *static_cast<const DataTypeNullable &>(type).getNestedType() : type;
|
||||||
|
column_holder = holder_type.createColumn()->cloneResized(numSpecialValues());
|
||||||
|
index.setColumn(getRawColumnPtr());
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
ColumnUnique<ColumnType>::ColumnUnique(MutableColumnPtr && holder, bool is_nullable)
|
||||||
|
: column_holder(std::move(holder))
|
||||||
|
, is_nullable(is_nullable)
|
||||||
|
, index(numSpecialValues(is_nullable), 0)
|
||||||
|
{
|
||||||
|
if (column_holder->size() < numSpecialValues())
|
||||||
|
throw Exception("Too small holder column for ColumnUnique.", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
if (column_holder->isColumnNullable())
|
||||||
|
throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
|
index.setColumn(getRawColumnPtr());
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
const ColumnPtr & ColumnUnique<ColumnType>::getNestedColumn() const
|
||||||
|
{
|
||||||
|
if (is_nullable)
|
||||||
|
{
|
||||||
|
size_t size = getRawColumnPtr()->size();
|
||||||
|
if (!cached_null_mask)
|
||||||
|
{
|
||||||
|
ColumnUInt8::MutablePtr null_mask = ColumnUInt8::create(size, UInt8(0));
|
||||||
|
null_mask->getData()[getNullValueIndex()] = 1;
|
||||||
|
cached_null_mask = std::move(null_mask);
|
||||||
|
cached_column_nullable = ColumnNullable::create(column_holder, cached_null_mask);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cached_null_mask->size() != size)
|
||||||
|
{
|
||||||
|
MutableColumnPtr null_mask = (*std::move(cached_null_mask)).mutate();
|
||||||
|
static_cast<ColumnUInt8 &>(*null_mask).getData().resize_fill(size);
|
||||||
|
cached_null_mask = std::move(null_mask);
|
||||||
|
cached_column_nullable = ColumnNullable::create(column_holder, cached_null_mask);
|
||||||
|
}
|
||||||
|
|
||||||
|
return cached_column_nullable;
|
||||||
|
}
|
||||||
|
return column_holder;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
size_t ColumnUnique<ColumnType>::getNullValueIndex() const
|
||||||
|
{
|
||||||
|
if (!is_nullable)
|
||||||
|
throw Exception("ColumnUnique can't contain null values.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
|
||||||
|
{
|
||||||
|
if (x.getType() == Field::Types::Null)
|
||||||
|
return getNullValueIndex();
|
||||||
|
|
||||||
|
auto column = getRawColumnPtr();
|
||||||
|
auto prev_size = static_cast<UInt64>(column->size());
|
||||||
|
|
||||||
|
if ((*column)[getDefaultValueIndex()] == x)
|
||||||
|
return getDefaultValueIndex();
|
||||||
|
|
||||||
|
column->insert(x);
|
||||||
|
auto pos = index.insert(prev_size);
|
||||||
|
if (pos != prev_size)
|
||||||
|
column->popBack(1);
|
||||||
|
|
||||||
|
return pos;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
size_t ColumnUnique<ColumnType>::uniqueInsertFrom(const IColumn & src, size_t n)
|
||||||
|
{
|
||||||
|
if (is_nullable && src.isNullAt(n))
|
||||||
|
return getNullValueIndex();
|
||||||
|
|
||||||
|
if (auto * nullable = typeid_cast<const ColumnNullable *>(&src))
|
||||||
|
return uniqueInsertFrom(nullable->getNestedColumn(), n);
|
||||||
|
|
||||||
|
auto ref = src.getDataAt(n);
|
||||||
|
return uniqueInsertData(ref.data, ref.size);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
size_t ColumnUnique<ColumnType>::uniqueInsertData(const char * pos, size_t length)
|
||||||
|
{
|
||||||
|
auto column = getRawColumnPtr();
|
||||||
|
|
||||||
|
if (column->getDataAt(getDefaultValueIndex()) == StringRef(pos, length))
|
||||||
|
return getDefaultValueIndex();
|
||||||
|
|
||||||
|
UInt64 size = column->size();
|
||||||
|
UInt64 insertion_point = index.getInsertionPoint(StringRef(pos, length));
|
||||||
|
|
||||||
|
if (insertion_point == size)
|
||||||
|
{
|
||||||
|
column->insertData(pos, length);
|
||||||
|
index.insertFromLastRow();
|
||||||
|
}
|
||||||
|
|
||||||
|
return insertion_point;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
size_t ColumnUnique<ColumnType>::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length)
|
||||||
|
{
|
||||||
|
if (std::is_same<ColumnType, ColumnString>::value)
|
||||||
|
return uniqueInsertData(pos, length - 1);
|
||||||
|
|
||||||
|
if (column_holder->valuesHaveFixedSize())
|
||||||
|
return uniqueInsertData(pos, length);
|
||||||
|
|
||||||
|
/// Don't know if data actually has terminating zero. So, insert it firstly.
|
||||||
|
|
||||||
|
auto column = getRawColumnPtr();
|
||||||
|
size_t prev_size = column->size();
|
||||||
|
column->insertDataWithTerminatingZero(pos, length);
|
||||||
|
|
||||||
|
if (column->compareAt(getDefaultValueIndex(), prev_size, *column, 1) == 0)
|
||||||
|
{
|
||||||
|
column->popBack(1);
|
||||||
|
return getDefaultValueIndex();
|
||||||
|
}
|
||||||
|
|
||||||
|
auto position = index.insert(prev_size);
|
||||||
|
if (position != prev_size)
|
||||||
|
column->popBack(1);
|
||||||
|
|
||||||
|
return static_cast<size_t>(position);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
size_t ColumnUnique<ColumnType>::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos)
|
||||||
|
{
|
||||||
|
auto column = getRawColumnPtr();
|
||||||
|
size_t prev_size = column->size();
|
||||||
|
new_pos = column->deserializeAndInsertFromArena(pos);
|
||||||
|
|
||||||
|
if (column->compareAt(getDefaultValueIndex(), prev_size, *column, 1) == 0)
|
||||||
|
{
|
||||||
|
column->popBack(1);
|
||||||
|
return getDefaultValueIndex();
|
||||||
|
}
|
||||||
|
|
||||||
|
auto index_pos = index.insert(prev_size);
|
||||||
|
if (index_pos != prev_size)
|
||||||
|
column->popBack(1);
|
||||||
|
|
||||||
|
return static_cast<size_t>(index_pos);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename IndexType>
|
||||||
|
static void checkIndexes(const ColumnVector<IndexType> & indexes, size_t max_dictionary_size)
|
||||||
|
{
|
||||||
|
auto & data = indexes.getData();
|
||||||
|
for (size_t i = 0; i < data.size(); ++i)
|
||||||
|
{
|
||||||
|
if (data[i] >= max_dictionary_size)
|
||||||
|
{
|
||||||
|
throw Exception("Found index " + toString(data[i]) + " at position " + toString(i)
|
||||||
|
+ " which is grated or equal than dictionary size " + toString(max_dictionary_size),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
template <typename IndexType>
|
||||||
|
MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
||||||
|
const IColumn & src,
|
||||||
|
size_t start,
|
||||||
|
size_t length,
|
||||||
|
size_t num_added_rows,
|
||||||
|
typename ColumnVector<IndexType>::MutablePtr && positions_column,
|
||||||
|
ReverseIndex<UInt64, ColumnType> * secondary_index,
|
||||||
|
size_t max_dictionary_size)
|
||||||
|
{
|
||||||
|
const ColumnType * src_column;
|
||||||
|
const NullMap * null_map = nullptr;
|
||||||
|
auto & positions = positions_column->getData();
|
||||||
|
|
||||||
|
auto update_position = [&](UInt64 & next_position) -> MutableColumnPtr
|
||||||
|
{
|
||||||
|
constexpr auto next_size = NumberTraits::nextSize(sizeof(IndexType));
|
||||||
|
using SuperiorIndexType = typename NumberTraits::Construct<false, false, next_size>::Type;
|
||||||
|
|
||||||
|
++next_position;
|
||||||
|
|
||||||
|
if (next_position > std::numeric_limits<IndexType>::max())
|
||||||
|
{
|
||||||
|
if (sizeof(SuperiorIndexType) == sizeof(IndexType))
|
||||||
|
throw Exception("Can't find superior index type for type " + demangle(typeid(IndexType).name()),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto expanded_column = ColumnVector<SuperiorIndexType>::create(length);
|
||||||
|
auto & expanded_data = expanded_column->getData();
|
||||||
|
for (size_t i = 0; i < num_added_rows; ++i)
|
||||||
|
expanded_data[i] = positions[i];
|
||||||
|
|
||||||
|
return uniqueInsertRangeImpl<SuperiorIndexType>(
|
||||||
|
src,
|
||||||
|
start,
|
||||||
|
length,
|
||||||
|
num_added_rows,
|
||||||
|
std::move(expanded_column),
|
||||||
|
secondary_index,
|
||||||
|
max_dictionary_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
return nullptr;
|
||||||
|
};
|
||||||
|
|
||||||
|
if (auto nullable_column = typeid_cast<const ColumnNullable *>(&src))
|
||||||
|
{
|
||||||
|
src_column = typeid_cast<const ColumnType *>(&nullable_column->getNestedColumn());
|
||||||
|
null_map = &nullable_column->getNullMapData();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
src_column = typeid_cast<const ColumnType *>(&src);
|
||||||
|
|
||||||
|
if (src_column == nullptr)
|
||||||
|
throw Exception("Invalid column type for ColumnUnique::insertRangeFrom. Expected " + column_holder->getName() +
|
||||||
|
", got " + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
|
auto column = getRawColumnPtr();
|
||||||
|
|
||||||
|
UInt64 next_position = column->size();
|
||||||
|
if (secondary_index)
|
||||||
|
next_position += secondary_index->size();
|
||||||
|
|
||||||
|
auto check_inserted_position = [&next_position](UInt64 inserted_position)
|
||||||
|
{
|
||||||
|
if (inserted_position != next_position)
|
||||||
|
throw Exception("Inserted position " + toString(inserted_position)
|
||||||
|
+ " is not equal with expected " + toString(next_position), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
};
|
||||||
|
|
||||||
|
auto insert_key = [&](const StringRef & ref, ReverseIndex<UInt64, ColumnType> * cur_index)
|
||||||
|
{
|
||||||
|
positions[num_added_rows] = next_position;
|
||||||
|
cur_index->getColumn()->insertData(ref.data, ref.size);
|
||||||
|
auto inserted_pos = cur_index->insertFromLastRow();
|
||||||
|
check_inserted_position(inserted_pos);
|
||||||
|
return update_position(next_position);
|
||||||
|
};
|
||||||
|
|
||||||
|
for (; num_added_rows < length; ++num_added_rows)
|
||||||
|
{
|
||||||
|
auto row = start + num_added_rows;
|
||||||
|
|
||||||
|
if (null_map && (*null_map)[row])
|
||||||
|
positions[num_added_rows] = getNullValueIndex();
|
||||||
|
else if (column->compareAt(getDefaultValueIndex(), row, *src_column, 1) == 0)
|
||||||
|
positions[num_added_rows] = getDefaultValueIndex();
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto ref = src_column->getDataAt(row);
|
||||||
|
auto cur_index = &index;
|
||||||
|
bool inserted = false;
|
||||||
|
|
||||||
|
while (!inserted)
|
||||||
|
{
|
||||||
|
auto insertion_point = cur_index->getInsertionPoint(ref);
|
||||||
|
|
||||||
|
if (insertion_point == cur_index->lastInsertionPoint())
|
||||||
|
{
|
||||||
|
if (secondary_index && cur_index != secondary_index && next_position >= max_dictionary_size)
|
||||||
|
{
|
||||||
|
cur_index = secondary_index;
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (auto res = insert_key(ref, cur_index))
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
positions[num_added_rows] = insertion_point;
|
||||||
|
|
||||||
|
inserted = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// checkIndexes(*positions_column, column->size() + (overflowed_keys ? overflowed_keys->size() : 0));
|
||||||
|
return std::move(positions_column);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length)
|
||||||
|
{
|
||||||
|
auto callForType = [this, &src, start, length](auto x) -> MutableColumnPtr
|
||||||
|
{
|
||||||
|
size_t size = getRawColumnPtr()->size();
|
||||||
|
|
||||||
|
using IndexType = decltype(x);
|
||||||
|
if (size <= std::numeric_limits<IndexType>::max())
|
||||||
|
{
|
||||||
|
auto positions = ColumnVector<IndexType>::create(length);
|
||||||
|
return this->uniqueInsertRangeImpl<IndexType>(src, start, length, 0, std::move(positions), nullptr, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
return nullptr;
|
||||||
|
};
|
||||||
|
|
||||||
|
MutableColumnPtr positions_column;
|
||||||
|
if (!positions_column)
|
||||||
|
positions_column = callForType(UInt8());
|
||||||
|
if (!positions_column)
|
||||||
|
positions_column = callForType(UInt16());
|
||||||
|
if (!positions_column)
|
||||||
|
positions_column = callForType(UInt32());
|
||||||
|
if (!positions_column)
|
||||||
|
positions_column = callForType(UInt64());
|
||||||
|
if (!positions_column)
|
||||||
|
throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
return positions_column;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType>::uniqueInsertRangeWithOverflow(
|
||||||
|
const IColumn & src,
|
||||||
|
size_t start,
|
||||||
|
size_t length,
|
||||||
|
size_t max_dictionary_size)
|
||||||
|
{
|
||||||
|
auto overflowed_keys = column_holder->cloneEmpty();
|
||||||
|
auto overflowed_keys_ptr = typeid_cast<ColumnType *>(overflowed_keys.get());
|
||||||
|
if (!overflowed_keys_ptr)
|
||||||
|
throw Exception("Invalid keys type for ColumnUnique.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto callForType = [this, &src, start, length, overflowed_keys_ptr, max_dictionary_size](auto x) -> MutableColumnPtr
|
||||||
|
{
|
||||||
|
size_t size = getRawColumnPtr()->size();
|
||||||
|
|
||||||
|
using IndexType = decltype(x);
|
||||||
|
if (size <= std::numeric_limits<IndexType>::max())
|
||||||
|
{
|
||||||
|
auto positions = ColumnVector<IndexType>::create(length);
|
||||||
|
ReverseIndex<UInt64, ColumnType> secondary_index(0, max_dictionary_size);
|
||||||
|
secondary_index.setColumn(overflowed_keys_ptr);
|
||||||
|
return this->uniqueInsertRangeImpl<IndexType>(src, start, length, 0, std::move(positions),
|
||||||
|
&secondary_index, max_dictionary_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
return nullptr;
|
||||||
|
};
|
||||||
|
|
||||||
|
MutableColumnPtr positions_column;
|
||||||
|
if (!positions_column)
|
||||||
|
positions_column = callForType(UInt8());
|
||||||
|
if (!positions_column)
|
||||||
|
positions_column = callForType(UInt16());
|
||||||
|
if (!positions_column)
|
||||||
|
positions_column = callForType(UInt32());
|
||||||
|
if (!positions_column)
|
||||||
|
positions_column = callForType(UInt64());
|
||||||
|
if (!positions_column)
|
||||||
|
throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
IColumnUnique::IndexesWithOverflow indexes_with_overflow;
|
||||||
|
indexes_with_overflow.indexes = std::move(positions_column);
|
||||||
|
indexes_with_overflow.overflowed_keys = std::move(overflowed_keys);
|
||||||
|
return indexes_with_overflow;
|
||||||
|
}
|
||||||
|
|
||||||
|
};
|
@ -17,6 +17,8 @@
|
|||||||
|
|
||||||
#if __SSE2__
|
#if __SSE2__
|
||||||
#include <emmintrin.h>
|
#include <emmintrin.h>
|
||||||
|
#include <Columns/ColumnsCommon.h>
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
@ -230,6 +232,12 @@ ColumnPtr ColumnVector<T>::permute(const IColumn::Permutation & perm, size_t lim
|
|||||||
return std::move(res);
|
return std::move(res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
ColumnPtr ColumnVector<T>::index(const IColumn & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
return selectIndexImpl(*this, indexes, limit);
|
||||||
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
ColumnPtr ColumnVector<T>::replicate(const IColumn::Offsets & offsets) const
|
ColumnPtr ColumnVector<T>::replicate(const IColumn::Offsets & offsets) const
|
||||||
{
|
{
|
||||||
|
@ -252,6 +252,11 @@ public:
|
|||||||
|
|
||||||
ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override;
|
ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override;
|
||||||
|
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
|
||||||
|
|
||||||
|
template <typename Type>
|
||||||
|
ColumnPtr indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const;
|
||||||
|
|
||||||
ColumnPtr replicate(const IColumn::Offsets & offsets) const override;
|
ColumnPtr replicate(const IColumn::Offsets & offsets) const override;
|
||||||
|
|
||||||
void getExtremes(Field & min, Field & max) const override;
|
void getExtremes(Field & min, Field & max) const override;
|
||||||
@ -295,5 +300,23 @@ protected:
|
|||||||
Container data;
|
Container data;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
template <typename Type>
|
||||||
|
ColumnPtr ColumnVector<T>::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
|
||||||
|
{
|
||||||
|
size_t size = indexes.size();
|
||||||
|
|
||||||
|
if (limit == 0)
|
||||||
|
limit = size;
|
||||||
|
else
|
||||||
|
limit = std::min(size, limit);
|
||||||
|
|
||||||
|
auto res = this->create(limit);
|
||||||
|
typename Self::Container & res_data = res->getData();
|
||||||
|
for (size_t i = 0; i < limit; ++i)
|
||||||
|
res_data[i] = data[indexes[i]];
|
||||||
|
|
||||||
|
return std::move(res);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
619
dbms/src/Columns/ColumnWithDictionary.cpp
Normal file
619
dbms/src/Columns/ColumnWithDictionary.cpp
Normal file
@ -0,0 +1,619 @@
|
|||||||
|
#include <Columns/ColumnWithDictionary.h>
|
||||||
|
#include <Columns/ColumnsNumber.h>
|
||||||
|
#include <DataStreams/ColumnGathererStream.h>
|
||||||
|
#include <DataTypes/NumberTraits.h>
|
||||||
|
#include <Common/HashTable/HashMap.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
template <typename T>
|
||||||
|
PaddedPODArray<T> * getIndexesData(IColumn & indexes)
|
||||||
|
{
|
||||||
|
auto * column = typeid_cast<ColumnVector<T> *>(&indexes);
|
||||||
|
if (column)
|
||||||
|
return &column->getData();
|
||||||
|
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
MutableColumnPtr mapUniqueIndexImplRef(PaddedPODArray<T> & index)
|
||||||
|
{
|
||||||
|
PaddedPODArray<T> copy(index.cbegin(), index.cend());
|
||||||
|
|
||||||
|
HashMap<T, T> hash_map;
|
||||||
|
for (auto val : index)
|
||||||
|
hash_map.insert({val, hash_map.size()});
|
||||||
|
|
||||||
|
auto res_col = ColumnVector<T>::create();
|
||||||
|
auto & data = res_col->getData();
|
||||||
|
|
||||||
|
data.resize(hash_map.size());
|
||||||
|
for (auto val : hash_map)
|
||||||
|
data[val.second] = val.first;
|
||||||
|
|
||||||
|
for (auto & ind : index)
|
||||||
|
ind = hash_map[ind];
|
||||||
|
|
||||||
|
for (size_t i = 0; i < index.size(); ++i)
|
||||||
|
if (data[index[i]] != copy[i])
|
||||||
|
throw Exception("Expected " + toString(data[index[i]]) + ", but got " + toString(copy[i]), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
return std::move(res_col);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
MutableColumnPtr mapUniqueIndexImpl(PaddedPODArray<T> & index)
|
||||||
|
{
|
||||||
|
if (index.empty())
|
||||||
|
return ColumnVector<T>::create();
|
||||||
|
|
||||||
|
auto size = index.size();
|
||||||
|
|
||||||
|
T max_val = index[0];
|
||||||
|
for (size_t i = 1; i < size; ++i)
|
||||||
|
max_val = std::max(max_val, index[i]);
|
||||||
|
|
||||||
|
/// May happen when dictionary is shared.
|
||||||
|
if (max_val > size)
|
||||||
|
return mapUniqueIndexImplRef(index);
|
||||||
|
|
||||||
|
auto map_size = UInt64(max_val) + 1;
|
||||||
|
PaddedPODArray<T> map(map_size, 0);
|
||||||
|
T zero_pos_value = index[0];
|
||||||
|
index[0] = 0;
|
||||||
|
T cur_pos = 0;
|
||||||
|
for (size_t i = 1; i < size; ++i)
|
||||||
|
{
|
||||||
|
T val = index[i];
|
||||||
|
if (val != zero_pos_value && map[val] == 0)
|
||||||
|
{
|
||||||
|
++cur_pos;
|
||||||
|
map[val] = cur_pos;
|
||||||
|
}
|
||||||
|
|
||||||
|
index[i] = map[val];
|
||||||
|
}
|
||||||
|
|
||||||
|
auto res_col = ColumnVector<T>::create(UInt64(cur_pos) + 1);
|
||||||
|
auto & data = res_col->getData();
|
||||||
|
data[0] = zero_pos_value;
|
||||||
|
for (size_t i = 0; i < map_size; ++i)
|
||||||
|
{
|
||||||
|
auto val = map[i];
|
||||||
|
if (val)
|
||||||
|
data[val] = static_cast<T>(i);
|
||||||
|
}
|
||||||
|
|
||||||
|
return std::move(res_col);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returns unique values of column. Write new index to column.
|
||||||
|
MutableColumnPtr mapUniqueIndex(IColumn & column)
|
||||||
|
{
|
||||||
|
if (auto * data_uint8 = getIndexesData<UInt8>(column))
|
||||||
|
return mapUniqueIndexImpl(*data_uint8);
|
||||||
|
else if (auto * data_uint16 = getIndexesData<UInt16>(column))
|
||||||
|
return mapUniqueIndexImpl(*data_uint16);
|
||||||
|
else if (auto * data_uint32 = getIndexesData<UInt32>(column))
|
||||||
|
return mapUniqueIndexImpl(*data_uint32);
|
||||||
|
else if (auto * data_uint64 = getIndexesData<UInt64>(column))
|
||||||
|
return mapUniqueIndexImpl(*data_uint64);
|
||||||
|
else
|
||||||
|
throw Exception("Indexes column for getUniqueIndex must be ColumnUInt, got" + column.getName(),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_)
|
||||||
|
: dictionary(std::move(column_unique_)), idx(std::move(indexes_))
|
||||||
|
{
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::insert(const Field & x)
|
||||||
|
{
|
||||||
|
compactIfSharedDictionary();
|
||||||
|
idx.insertPosition(dictionary.getColumnUnique().uniqueInsert(x));
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::insertDefault()
|
||||||
|
{
|
||||||
|
idx.insertPosition(getDictionary().getDefaultValueIndex());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::insertFrom(const IColumn & src, size_t n)
|
||||||
|
{
|
||||||
|
auto * src_with_dict = typeid_cast<const ColumnWithDictionary *>(&src);
|
||||||
|
|
||||||
|
if (!src_with_dict)
|
||||||
|
throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
|
size_t position = src_with_dict->getIndexes().getUInt(n);
|
||||||
|
|
||||||
|
if (&src_with_dict->getDictionary() == &getDictionary())
|
||||||
|
{
|
||||||
|
/// Dictionary is shared with src column. Insert only index.
|
||||||
|
idx.insertPosition(position);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
compactIfSharedDictionary();
|
||||||
|
const auto & nested = *src_with_dict->getDictionary().getNestedColumn();
|
||||||
|
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(nested, position));
|
||||||
|
}
|
||||||
|
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::insertFromFullColumn(const IColumn & src, size_t n)
|
||||||
|
{
|
||||||
|
compactIfSharedDictionary();
|
||||||
|
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(src, n));
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::insertRangeFrom(const IColumn & src, size_t start, size_t length)
|
||||||
|
{
|
||||||
|
auto * src_with_dict = typeid_cast<const ColumnWithDictionary *>(&src);
|
||||||
|
|
||||||
|
if (!src_with_dict)
|
||||||
|
throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
|
if (&src_with_dict->getDictionary() == &getDictionary())
|
||||||
|
{
|
||||||
|
/// Dictionary is shared with src column. Insert only indexes.
|
||||||
|
idx.insertPositionsRange(src_with_dict->getIndexes(), start, length);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
compactIfSharedDictionary();
|
||||||
|
|
||||||
|
/// TODO: Support native insertion from other unique column. It will help to avoid null map creation.
|
||||||
|
|
||||||
|
auto sub_idx = (*src_with_dict->getIndexes().cut(start, length)).mutate();
|
||||||
|
auto idx_map = mapUniqueIndex(*sub_idx);
|
||||||
|
|
||||||
|
auto src_nested = src_with_dict->getDictionary().getNestedColumn();
|
||||||
|
auto used_keys = src_nested->index(*idx_map, 0);
|
||||||
|
|
||||||
|
auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(*used_keys, 0, used_keys->size());
|
||||||
|
idx.insertPositionsRange(*inserted_indexes->index(*sub_idx, 0), 0, length);
|
||||||
|
}
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length)
|
||||||
|
{
|
||||||
|
compactIfSharedDictionary();
|
||||||
|
auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(src, start, length);
|
||||||
|
idx.insertPositionsRange(*inserted_indexes, 0, length);
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions)
|
||||||
|
{
|
||||||
|
Index(positions.getPtr()).check(keys.size());
|
||||||
|
compactIfSharedDictionary();
|
||||||
|
auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(keys, 0, keys.size());
|
||||||
|
idx.insertPositionsRange(*inserted_indexes->index(positions, 0), 0, positions.size());
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::insertData(const char * pos, size_t length)
|
||||||
|
{
|
||||||
|
compactIfSharedDictionary();
|
||||||
|
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertData(pos, length));
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::insertDataWithTerminatingZero(const char * pos, size_t length)
|
||||||
|
{
|
||||||
|
compactIfSharedDictionary();
|
||||||
|
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertDataWithTerminatingZero(pos, length));
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
StringRef ColumnWithDictionary::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
|
||||||
|
{
|
||||||
|
return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin);
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * ColumnWithDictionary::deserializeAndInsertFromArena(const char * pos)
|
||||||
|
{
|
||||||
|
compactIfSharedDictionary();
|
||||||
|
|
||||||
|
const char * new_pos;
|
||||||
|
idx.insertPosition(dictionary.getColumnUnique().uniqueDeserializeAndInsertFromArena(pos, new_pos));
|
||||||
|
|
||||||
|
idx.check(getDictionary().size());
|
||||||
|
return new_pos;
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::gather(ColumnGathererStream & gatherer)
|
||||||
|
{
|
||||||
|
gatherer.gather(*this);
|
||||||
|
}
|
||||||
|
|
||||||
|
MutableColumnPtr ColumnWithDictionary::cloneResized(size_t size) const
|
||||||
|
{
|
||||||
|
auto unique_ptr = dictionary.getColumnUniquePtr();
|
||||||
|
return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), getIndexes().cloneResized(size));
|
||||||
|
}
|
||||||
|
|
||||||
|
int ColumnWithDictionary::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const
|
||||||
|
{
|
||||||
|
const auto & column_with_dictionary = static_cast<const ColumnWithDictionary &>(rhs);
|
||||||
|
size_t n_index = getIndexes().getUInt(n);
|
||||||
|
size_t m_index = column_with_dictionary.getIndexes().getUInt(m);
|
||||||
|
return getDictionary().compareAt(n_index, m_index, column_with_dictionary.getDictionary(), nan_direction_hint);
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
|
||||||
|
{
|
||||||
|
if (limit == 0)
|
||||||
|
limit = size();
|
||||||
|
|
||||||
|
size_t unique_limit = std::min(limit, getDictionary().size());
|
||||||
|
Permutation unique_perm;
|
||||||
|
getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm);
|
||||||
|
|
||||||
|
/// TODO: optimize with sse.
|
||||||
|
|
||||||
|
/// Get indexes per row in column_unique.
|
||||||
|
std::vector<std::vector<size_t>> indexes_per_row(getDictionary().size());
|
||||||
|
size_t indexes_size = getIndexes().size();
|
||||||
|
for (size_t row = 0; row < indexes_size; ++row)
|
||||||
|
indexes_per_row[getIndexes().getUInt(row)].push_back(row);
|
||||||
|
|
||||||
|
/// Replicate permutation.
|
||||||
|
size_t perm_size = std::min(indexes_size, limit);
|
||||||
|
res.resize(perm_size);
|
||||||
|
size_t perm_index = 0;
|
||||||
|
for (size_t row = 0; row < indexes_size && perm_index < perm_size; ++row)
|
||||||
|
{
|
||||||
|
const auto & row_indexes = indexes_per_row[unique_perm[row]];
|
||||||
|
for (auto row_index : row_indexes)
|
||||||
|
{
|
||||||
|
res[perm_index] = row_index;
|
||||||
|
++perm_index;
|
||||||
|
|
||||||
|
if (perm_index == perm_size)
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<MutableColumnPtr> ColumnWithDictionary::scatter(ColumnIndex num_columns, const Selector & selector) const
|
||||||
|
{
|
||||||
|
auto columns = getIndexes().scatter(num_columns, selector);
|
||||||
|
for (auto & column : columns)
|
||||||
|
{
|
||||||
|
auto unique_ptr = dictionary.getColumnUniquePtr();
|
||||||
|
column = ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), std::move(column));
|
||||||
|
}
|
||||||
|
|
||||||
|
return columns;
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::setSharedDictionary(const ColumnPtr & column_unique)
|
||||||
|
{
|
||||||
|
if (!empty())
|
||||||
|
throw Exception("Can't set ColumnUnique for ColumnWithDictionary because is't not empty.",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
dictionary.setShared(column_unique);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnWithDictionary::MutablePtr ColumnWithDictionary::compact()
|
||||||
|
{
|
||||||
|
auto positions = idx.getPositions();
|
||||||
|
/// Create column with new indexes and old dictionary.
|
||||||
|
auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), (*std::move(positions)).mutate());
|
||||||
|
/// Will create new dictionary.
|
||||||
|
column->compactInplace();
|
||||||
|
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnWithDictionary::MutablePtr ColumnWithDictionary::cutAndCompact(size_t start, size_t length) const
|
||||||
|
{
|
||||||
|
auto sub_positions = (*idx.getPositions()->cut(start, length)).mutate();
|
||||||
|
/// Create column with new indexes and old dictionary.
|
||||||
|
auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), std::move(sub_positions));
|
||||||
|
/// Will create new dictionary.
|
||||||
|
column->compactInplace();
|
||||||
|
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::compactInplace()
|
||||||
|
{
|
||||||
|
auto positions = idx.detachPositions();
|
||||||
|
dictionary.compact(positions);
|
||||||
|
idx.attachPositions(std::move(positions));
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::compactIfSharedDictionary()
|
||||||
|
{
|
||||||
|
if (dictionary.isShared())
|
||||||
|
compactInplace();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnWithDictionary::DictionaryEncodedColumn
|
||||||
|
ColumnWithDictionary::getMinimalDictionaryEncodedColumn(size_t offset, size_t limit) const
|
||||||
|
{
|
||||||
|
MutableColumnPtr sub_indexes = (*std::move(idx.getPositions()->cut(offset, limit))).mutate();
|
||||||
|
auto indexes_map = mapUniqueIndex(*sub_indexes);
|
||||||
|
auto sub_keys = getDictionary().getNestedColumn()->index(*indexes_map, 0);
|
||||||
|
|
||||||
|
return {std::move(sub_keys), std::move(sub_indexes)};
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnWithDictionary::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {}
|
||||||
|
|
||||||
|
ColumnWithDictionary::Index::Index(MutableColumnPtr && positions) : positions(std::move(positions))
|
||||||
|
{
|
||||||
|
updateSizeOfType();
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnWithDictionary::Index::Index(ColumnPtr positions) : positions(std::move(positions))
|
||||||
|
{
|
||||||
|
updateSizeOfType();
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename Callback>
|
||||||
|
void ColumnWithDictionary::Index::callForType(Callback && callback, size_t size_of_type)
|
||||||
|
{
|
||||||
|
switch (size_of_type)
|
||||||
|
{
|
||||||
|
case sizeof(UInt8): { callback(UInt8()); break; }
|
||||||
|
case sizeof(UInt16): { callback(UInt16()); break; }
|
||||||
|
case sizeof(UInt32): { callback(UInt32()); break; }
|
||||||
|
case sizeof(UInt64): { callback(UInt64()); break; }
|
||||||
|
default: {
|
||||||
|
throw Exception("Unexpected size of index type for ColumnWithDictionary: " + toString(size_of_type),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t ColumnWithDictionary::Index::getSizeOfIndexType(const IColumn & column, size_t hint)
|
||||||
|
{
|
||||||
|
auto checkFor = [&](auto type) { return typeid_cast<const ColumnVector<decltype(type)> *>(&column) != nullptr; };
|
||||||
|
auto tryGetSizeFor = [&](auto type) -> size_t { return checkFor(type) ? sizeof(decltype(type)) : 0; };
|
||||||
|
|
||||||
|
if (hint)
|
||||||
|
{
|
||||||
|
size_t size = 0;
|
||||||
|
callForType([&](auto type) { size = tryGetSizeFor(type); }, hint);
|
||||||
|
|
||||||
|
if (size)
|
||||||
|
return size;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (auto size = tryGetSizeFor(UInt8()))
|
||||||
|
return size;
|
||||||
|
if (auto size = tryGetSizeFor(UInt16()))
|
||||||
|
return size;
|
||||||
|
if (auto size = tryGetSizeFor(UInt32()))
|
||||||
|
return size;
|
||||||
|
if (auto size = tryGetSizeFor(UInt64()))
|
||||||
|
return size;
|
||||||
|
|
||||||
|
throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected UInt, got " + column.getName(),
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::Index::attachPositions(ColumnPtr positions_)
|
||||||
|
{
|
||||||
|
positions = std::move(positions_);
|
||||||
|
updateSizeOfType();
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename IndexType>
|
||||||
|
typename ColumnVector<IndexType>::Container & ColumnWithDictionary::Index::getPositionsData()
|
||||||
|
{
|
||||||
|
auto * positions_ptr = typeid_cast<ColumnVector<IndexType> *>(positions->assumeMutable().get());
|
||||||
|
if (!positions_ptr)
|
||||||
|
throw Exception("Invalid indexes type for ColumnWithDictionary."
|
||||||
|
" Expected UInt" + toString(8 * sizeof(IndexType)) + ", got " + positions->getName(),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
return positions_ptr->getData();
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename IndexType>
|
||||||
|
void ColumnWithDictionary::Index::convertPositions()
|
||||||
|
{
|
||||||
|
auto convert = [&](auto x)
|
||||||
|
{
|
||||||
|
using CurIndexType = decltype(x);
|
||||||
|
auto & data = getPositionsData<CurIndexType>();
|
||||||
|
|
||||||
|
if (sizeof(CurIndexType) > sizeof(IndexType))
|
||||||
|
throw Exception("Converting indexes to smaller type: from " + toString(sizeof(CurIndexType)) +
|
||||||
|
" to " + toString(sizeof(IndexType)), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
if (sizeof(CurIndexType) != sizeof(IndexType))
|
||||||
|
{
|
||||||
|
size_t size = data.size();
|
||||||
|
auto new_positions = ColumnVector<IndexType>::create(size);
|
||||||
|
auto & new_data = new_positions->getData();
|
||||||
|
|
||||||
|
/// TODO: Optimize with SSE?
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
new_data[i] = data[i];
|
||||||
|
|
||||||
|
positions = std::move(new_positions);
|
||||||
|
size_of_type = sizeof(IndexType);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
callForType(std::move(convert), size_of_type);
|
||||||
|
|
||||||
|
checkSizeOfType();
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::Index::expandType()
|
||||||
|
{
|
||||||
|
auto expand = [&](auto type)
|
||||||
|
{
|
||||||
|
using CurIndexType = decltype(type);
|
||||||
|
constexpr auto next_size = NumberTraits::nextSize(sizeof(CurIndexType));
|
||||||
|
if (next_size == sizeof(CurIndexType))
|
||||||
|
throw Exception("Can't expand indexes type for ColumnWithDictionary from type: "
|
||||||
|
+ demangle(typeid(CurIndexType).name()), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
using NewIndexType = typename NumberTraits::Construct<false, false, next_size>::Type;
|
||||||
|
convertPositions<NewIndexType>();
|
||||||
|
};
|
||||||
|
|
||||||
|
callForType(std::move(expand), size_of_type);
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt64 ColumnWithDictionary::Index::getMaxPositionForCurrentType() const
|
||||||
|
{
|
||||||
|
UInt64 value = 0;
|
||||||
|
callForType([&](auto type) { value = std::numeric_limits<decltype(type)>::max(); }, size_of_type);
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::Index::insertPosition(UInt64 position)
|
||||||
|
{
|
||||||
|
while (position > getMaxPositionForCurrentType())
|
||||||
|
expandType();
|
||||||
|
|
||||||
|
positions->assumeMutableRef().insert(UInt64(position));
|
||||||
|
checkSizeOfType();
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::Index::insertPositionsRange(const IColumn & column, size_t offset, size_t limit)
|
||||||
|
{
|
||||||
|
auto insertForType = [&](auto type)
|
||||||
|
{
|
||||||
|
using ColumnType = decltype(type);
|
||||||
|
const auto * column_ptr = typeid_cast<const ColumnVector<ColumnType> *>(&column);
|
||||||
|
|
||||||
|
if (!column_ptr)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
if (size_of_type < sizeof(ColumnType))
|
||||||
|
convertPositions<ColumnType>();
|
||||||
|
|
||||||
|
if (size_of_type == sizeof(ColumnType))
|
||||||
|
positions->assumeMutableRef().insertRangeFrom(column, offset, limit);
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto copy = [&](auto cur_type)
|
||||||
|
{
|
||||||
|
using CurIndexType = decltype(cur_type);
|
||||||
|
auto & positions_data = getPositionsData<CurIndexType>();
|
||||||
|
const auto & column_data = column_ptr->getData();
|
||||||
|
|
||||||
|
size_t size = positions_data.size();
|
||||||
|
positions_data.resize(size + limit);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < limit; ++i)
|
||||||
|
positions_data[size + i] = column_data[offset + i];
|
||||||
|
};
|
||||||
|
|
||||||
|
callForType(std::move(copy), size_of_type);
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
};
|
||||||
|
|
||||||
|
if (!insertForType(UInt8()) &&
|
||||||
|
!insertForType(UInt16()) &&
|
||||||
|
!insertForType(UInt32()) &&
|
||||||
|
!insertForType(UInt64()))
|
||||||
|
throw Exception("Invalid column for ColumnWithDictionary index. Expected UInt, got " + column.getName(),
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
|
checkSizeOfType();
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::Index::check(size_t /*max_dictionary_size*/)
|
||||||
|
{
|
||||||
|
/// TODO: remove
|
||||||
|
/*
|
||||||
|
auto check = [&](auto cur_type)
|
||||||
|
{
|
||||||
|
using CurIndexType = decltype(cur_type);
|
||||||
|
auto & positions_data = getPositionsData<CurIndexType>();
|
||||||
|
|
||||||
|
for (size_t i = 0; i < positions_data.size(); ++i)
|
||||||
|
{
|
||||||
|
if (positions_data[i] >= max_dictionary_size)
|
||||||
|
{
|
||||||
|
throw Exception("Found index " + toString(positions_data[i]) + " at position " + toString(i)
|
||||||
|
+ " which is grated or equal than dictionary size " + toString(max_dictionary_size),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
callForType(std::move(check), size_of_type);
|
||||||
|
*/
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::Index::checkSizeOfType()
|
||||||
|
{
|
||||||
|
if (size_of_type != getSizeOfIndexType(*positions, size_of_type))
|
||||||
|
throw Exception("Invalid size of type. Expected " + toString(8 * size_of_type) +
|
||||||
|
", but positions are " + positions->getName(), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnWithDictionary::Dictionary::Dictionary(MutableColumnPtr && column_unique_)
|
||||||
|
: column_unique(std::move(column_unique_))
|
||||||
|
{
|
||||||
|
checkColumn(*column_unique);
|
||||||
|
}
|
||||||
|
ColumnWithDictionary::Dictionary::Dictionary(ColumnPtr column_unique_)
|
||||||
|
: column_unique(std::move(column_unique_))
|
||||||
|
{
|
||||||
|
checkColumn(*column_unique);
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::Dictionary::checkColumn(const IColumn & column)
|
||||||
|
{
|
||||||
|
|
||||||
|
if (!dynamic_cast<const IColumnUnique *>(&column))
|
||||||
|
throw Exception("ColumnUnique expected as an argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::Dictionary::setShared(const ColumnPtr & dictionary)
|
||||||
|
{
|
||||||
|
checkColumn(*dictionary);
|
||||||
|
|
||||||
|
column_unique = dictionary;
|
||||||
|
shared = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnWithDictionary::Dictionary::compact(ColumnPtr & positions)
|
||||||
|
{
|
||||||
|
auto new_column_unique = column_unique->cloneEmpty();
|
||||||
|
|
||||||
|
auto & unique = getColumnUnique();
|
||||||
|
auto & new_unique = static_cast<IColumnUnique &>(*new_column_unique);
|
||||||
|
|
||||||
|
auto indexes = mapUniqueIndex(positions->assumeMutableRef());
|
||||||
|
auto sub_keys = unique.getNestedColumn()->index(*indexes, 0);
|
||||||
|
auto new_indexes = new_unique.uniqueInsertRangeFrom(*sub_keys, 0, sub_keys->size());
|
||||||
|
|
||||||
|
positions = (*new_indexes->index(*positions, 0)).mutate();
|
||||||
|
column_unique = std::move(new_column_unique);
|
||||||
|
|
||||||
|
shared = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
248
dbms/src/Columns/ColumnWithDictionary.h
Normal file
248
dbms/src/Columns/ColumnWithDictionary.h
Normal file
@ -0,0 +1,248 @@
|
|||||||
|
#pragma once
|
||||||
|
#include <Columns/IColumn.h>
|
||||||
|
#include <Columns/IColumnUnique.h>
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <AggregateFunctions/AggregateFunctionCount.h>
|
||||||
|
#include "ColumnsNumber.h"
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
|
}
|
||||||
|
|
||||||
|
class ColumnWithDictionary final : public COWPtrHelper<IColumn, ColumnWithDictionary>
|
||||||
|
{
|
||||||
|
friend class COWPtrHelper<IColumn, ColumnWithDictionary>;
|
||||||
|
|
||||||
|
ColumnWithDictionary(MutableColumnPtr && column_unique, MutableColumnPtr && indexes);
|
||||||
|
ColumnWithDictionary(const ColumnWithDictionary & other) = default;
|
||||||
|
|
||||||
|
public:
|
||||||
|
/** Create immutable column using immutable arguments. This arguments may be shared with other columns.
|
||||||
|
* Use IColumn::mutate in order to make mutable column and mutate shared nested columns.
|
||||||
|
*/
|
||||||
|
using Base = COWPtrHelper<IColumn, ColumnWithDictionary>;
|
||||||
|
static Ptr create(const ColumnPtr & column_unique_, const ColumnPtr & indexes_)
|
||||||
|
{
|
||||||
|
return ColumnWithDictionary::create(column_unique_->assumeMutable(), indexes_->assumeMutable());
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ... Args, typename = typename std::enable_if<IsMutableColumns<Args ...>::value>::type>
|
||||||
|
static MutablePtr create(Args &&... args) { return Base::create(std::forward<Args>(args)...); }
|
||||||
|
|
||||||
|
|
||||||
|
std::string getName() const override { return "ColumnWithDictionary"; }
|
||||||
|
const char * getFamilyName() const override { return "ColumnWithDictionary"; }
|
||||||
|
|
||||||
|
ColumnPtr convertToFullColumn() const { return getDictionary().getNestedColumn()->index(getIndexes(), 0); }
|
||||||
|
ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); }
|
||||||
|
|
||||||
|
MutableColumnPtr cloneResized(size_t size) const override;
|
||||||
|
size_t size() const override { return getIndexes().size(); }
|
||||||
|
|
||||||
|
Field operator[](size_t n) const override { return getDictionary()[getIndexes().getUInt(n)]; }
|
||||||
|
void get(size_t n, Field & res) const override { getDictionary().get(getIndexes().getUInt(n), res); }
|
||||||
|
|
||||||
|
StringRef getDataAt(size_t n) const override { return getDictionary().getDataAt(getIndexes().getUInt(n)); }
|
||||||
|
StringRef getDataAtWithTerminatingZero(size_t n) const override
|
||||||
|
{
|
||||||
|
return getDictionary().getDataAtWithTerminatingZero(getIndexes().getUInt(n));
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt64 get64(size_t n) const override { return getDictionary().get64(getIndexes().getUInt(n)); }
|
||||||
|
UInt64 getUInt(size_t n) const override { return getDictionary().getUInt(getIndexes().getUInt(n)); }
|
||||||
|
Int64 getInt(size_t n) const override { return getDictionary().getInt(getIndexes().getUInt(n)); }
|
||||||
|
bool isNullAt(size_t n) const override { return getDictionary().isNullAt(getIndexes().getUInt(n)); }
|
||||||
|
ColumnPtr cut(size_t start, size_t length) const override
|
||||||
|
{
|
||||||
|
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().cut(start, length));
|
||||||
|
}
|
||||||
|
|
||||||
|
void insert(const Field & x) override;
|
||||||
|
void insertDefault() override;
|
||||||
|
|
||||||
|
void insertFrom(const IColumn & src, size_t n) override;
|
||||||
|
void insertFromFullColumn(const IColumn & src, size_t n);
|
||||||
|
|
||||||
|
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
||||||
|
void insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length);
|
||||||
|
void insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions);
|
||||||
|
|
||||||
|
void insertData(const char * pos, size_t length) override;
|
||||||
|
void insertDataWithTerminatingZero(const char * pos, size_t length) override;
|
||||||
|
|
||||||
|
|
||||||
|
void popBack(size_t n) override { idx.popBack(n); }
|
||||||
|
|
||||||
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char * pos) override;
|
||||||
|
|
||||||
|
void updateHashWithValue(size_t n, SipHash & hash) const override
|
||||||
|
{
|
||||||
|
return getDictionary().updateHashWithValue(getIndexes().getUInt(n), hash);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override
|
||||||
|
{
|
||||||
|
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint));
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override
|
||||||
|
{
|
||||||
|
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().permute(perm, limit));
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr index(const IColumn & indexes_, size_t limit) const override
|
||||||
|
{
|
||||||
|
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().index(indexes_, limit));
|
||||||
|
}
|
||||||
|
|
||||||
|
int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override;
|
||||||
|
|
||||||
|
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
|
||||||
|
|
||||||
|
ColumnPtr replicate(const Offsets & offsets) const override
|
||||||
|
{
|
||||||
|
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets));
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<MutableColumnPtr> scatter(ColumnIndex num_columns, const Selector & selector) const override;
|
||||||
|
|
||||||
|
void gather(ColumnGathererStream & gatherer_stream) override ;
|
||||||
|
void getExtremes(Field & min, Field & max) const override {
|
||||||
|
return getDictionary().index(getIndexes(), 0)->getExtremes(min, max); /// TODO: optimize
|
||||||
|
}
|
||||||
|
|
||||||
|
void reserve(size_t n) override { idx.reserve(n); }
|
||||||
|
|
||||||
|
size_t byteSize() const override { return idx.getPositions()->byteSize() + getDictionary().byteSize(); }
|
||||||
|
size_t allocatedBytes() const override { return idx.getPositions()->allocatedBytes() + getDictionary().allocatedBytes(); }
|
||||||
|
|
||||||
|
void forEachSubcolumn(ColumnCallback callback) override
|
||||||
|
{
|
||||||
|
callback(idx.getPositionsPtr());
|
||||||
|
|
||||||
|
/// Column doesn't own dictionary if it's shared.
|
||||||
|
if (!dictionary.isShared())
|
||||||
|
callback(dictionary.getColumnUniquePtr());
|
||||||
|
}
|
||||||
|
|
||||||
|
bool valuesHaveFixedSize() const override { return getDictionary().valuesHaveFixedSize(); }
|
||||||
|
bool isFixedAndContiguous() const override { return getDictionary().isFixedAndContiguous(); }
|
||||||
|
size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); }
|
||||||
|
bool isNumeric() const override { return getDictionary().isNumeric(); }
|
||||||
|
bool withDictionary() const override { return true; }
|
||||||
|
|
||||||
|
const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); }
|
||||||
|
/// IColumnUnique & getUnique() { return static_cast<IColumnUnique &>(*column_unique->assumeMutable()); }
|
||||||
|
/// ColumnPtr getUniquePtr() const { return column_unique; }
|
||||||
|
|
||||||
|
/// IColumn & getIndexes() { return idx.getPositions()->assumeMutableRef(); }
|
||||||
|
const IColumn & getIndexes() const { return *idx.getPositions(); }
|
||||||
|
const ColumnPtr & getIndexesPtr() const { return idx.getPositions(); }
|
||||||
|
|
||||||
|
///void setIndexes(MutableColumnPtr && indexes_) { indexes = std::move(indexes_); }
|
||||||
|
|
||||||
|
/// Set shared ColumnUnique for empty column with dictionary.
|
||||||
|
void setSharedDictionary(const ColumnPtr & column_unique);
|
||||||
|
|
||||||
|
/// Create column new dictionary with only keys that are mentioned in index.
|
||||||
|
MutablePtr compact();
|
||||||
|
|
||||||
|
/// Cut + compact.
|
||||||
|
MutablePtr cutAndCompact(size_t start, size_t length) const;
|
||||||
|
|
||||||
|
struct DictionaryEncodedColumn
|
||||||
|
{
|
||||||
|
ColumnPtr dictionary;
|
||||||
|
ColumnPtr indexes;
|
||||||
|
};
|
||||||
|
|
||||||
|
DictionaryEncodedColumn getMinimalDictionaryEncodedColumn(size_t offset, size_t limit) const;
|
||||||
|
|
||||||
|
class Index
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
Index();
|
||||||
|
Index(const Index & other) = default;
|
||||||
|
explicit Index(MutableColumnPtr && positions);
|
||||||
|
explicit Index(ColumnPtr positions);
|
||||||
|
|
||||||
|
const ColumnPtr & getPositions() const { return positions; }
|
||||||
|
ColumnPtr & getPositionsPtr() { return positions; }
|
||||||
|
void insertPosition(UInt64 position);
|
||||||
|
void insertPositionsRange(const IColumn & column, size_t offset, size_t limit);
|
||||||
|
|
||||||
|
void popBack(size_t n) { positions->assumeMutableRef().popBack(n); }
|
||||||
|
void reserve(size_t n) { positions->assumeMutableRef().reserve(n); }
|
||||||
|
|
||||||
|
UInt64 getMaxPositionForCurrentType() const;
|
||||||
|
|
||||||
|
static size_t getSizeOfIndexType(const IColumn & column, size_t hint);
|
||||||
|
|
||||||
|
void check(size_t max_dictionary_size);
|
||||||
|
void checkSizeOfType();
|
||||||
|
|
||||||
|
ColumnPtr detachPositions() { return std::move(positions); }
|
||||||
|
void attachPositions(ColumnPtr positions_);
|
||||||
|
|
||||||
|
private:
|
||||||
|
ColumnPtr positions;
|
||||||
|
size_t size_of_type = 0;
|
||||||
|
|
||||||
|
void updateSizeOfType() { size_of_type = getSizeOfIndexType(*positions, size_of_type); }
|
||||||
|
void expandType();
|
||||||
|
|
||||||
|
template <typename IndexType>
|
||||||
|
typename ColumnVector<IndexType>::Container & getPositionsData();
|
||||||
|
|
||||||
|
template <typename IndexType>
|
||||||
|
void convertPositions();
|
||||||
|
|
||||||
|
template <typename Callback>
|
||||||
|
static void callForType(Callback && callback, size_t size_of_type);
|
||||||
|
};
|
||||||
|
|
||||||
|
private:
|
||||||
|
class Dictionary
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
Dictionary(const Dictionary & other) = default;
|
||||||
|
explicit Dictionary(MutableColumnPtr && column_unique);
|
||||||
|
explicit Dictionary(ColumnPtr column_unique);
|
||||||
|
|
||||||
|
const ColumnPtr & getColumnUniquePtr() const { return column_unique; }
|
||||||
|
ColumnPtr & getColumnUniquePtr() { return column_unique; }
|
||||||
|
|
||||||
|
const IColumnUnique & getColumnUnique() const { return static_cast<const IColumnUnique &>(*column_unique); }
|
||||||
|
IColumnUnique & getColumnUnique() { return static_cast<IColumnUnique &>(column_unique->assumeMutableRef()); }
|
||||||
|
|
||||||
|
/// Dictionary may be shared for several mutable columns.
|
||||||
|
/// Immutable columns may have the same column unique, which isn't necessarily shared dictionary.
|
||||||
|
void setShared(const ColumnPtr & dictionary);
|
||||||
|
bool isShared() const { return shared; }
|
||||||
|
|
||||||
|
/// Create new dictionary with only keys that are mentioned in positions.
|
||||||
|
void compact(ColumnPtr & positions);
|
||||||
|
|
||||||
|
private:
|
||||||
|
ColumnPtr column_unique;
|
||||||
|
bool shared = false;
|
||||||
|
|
||||||
|
void checkColumn(const IColumn & column);
|
||||||
|
};
|
||||||
|
|
||||||
|
Dictionary dictionary;
|
||||||
|
Index idx;
|
||||||
|
|
||||||
|
void compactInplace();
|
||||||
|
void compactIfSharedDictionary();
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -3,6 +3,10 @@
|
|||||||
#endif
|
#endif
|
||||||
|
|
||||||
#include <Columns/IColumn.h>
|
#include <Columns/IColumn.h>
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Columns/ColumnVector.h>
|
||||||
|
#include <Common/HashTable/HashSet.h>
|
||||||
|
#include <Common/HashTable/HashMap.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -308,4 +312,22 @@ INSTANTIATE(Float64)
|
|||||||
|
|
||||||
#undef INSTANTIATE
|
#undef INSTANTIATE
|
||||||
|
|
||||||
|
namespace detail
|
||||||
|
{
|
||||||
|
template <typename T>
|
||||||
|
const PaddedPODArray<T> * getIndexesData(const IColumn & indexes)
|
||||||
|
{
|
||||||
|
auto * column = typeid_cast<const ColumnVector<T> *>(&indexes);
|
||||||
|
if (column)
|
||||||
|
return &column->getData();
|
||||||
|
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
template const PaddedPODArray<UInt8> * getIndexesData<UInt8>(const IColumn & indexes);
|
||||||
|
template const PaddedPODArray<UInt16> * getIndexesData<UInt16>(const IColumn & indexes);
|
||||||
|
template const PaddedPODArray<UInt32> * getIndexesData<UInt32>(const IColumn & indexes);
|
||||||
|
template const PaddedPODArray<UInt64> * getIndexesData<UInt64>(const IColumn & indexes);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -8,6 +8,11 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
}
|
||||||
|
|
||||||
/// Counts how many bytes of `filt` are greater than zero.
|
/// Counts how many bytes of `filt` are greater than zero.
|
||||||
size_t countBytesInFilter(const IColumn::Filter & filt);
|
size_t countBytesInFilter(const IColumn::Filter & filt);
|
||||||
|
|
||||||
@ -33,4 +38,38 @@ void filterArraysImplOnlyData(
|
|||||||
PaddedPODArray<T> & res_elems,
|
PaddedPODArray<T> & res_elems,
|
||||||
const IColumn::Filter & filt, ssize_t result_size_hint);
|
const IColumn::Filter & filt, ssize_t result_size_hint);
|
||||||
|
|
||||||
|
namespace detail
|
||||||
|
{
|
||||||
|
template <typename T>
|
||||||
|
const PaddedPODArray<T> * getIndexesData(const IColumn & indexes);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Check limit <= indexes->size() and call column.indexImpl(const PaddedPodArray<Type> & indexes, size_t limit).
|
||||||
|
template <typename Column>
|
||||||
|
ColumnPtr selectIndexImpl(const Column & column, const IColumn & indexes, size_t limit)
|
||||||
|
{
|
||||||
|
if (limit == 0)
|
||||||
|
limit = indexes.size();
|
||||||
|
|
||||||
|
if (indexes.size() < limit)
|
||||||
|
throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
if (auto * data_uint8 = detail::getIndexesData<UInt8>(indexes))
|
||||||
|
return column.template indexImpl<UInt8>(*data_uint8, limit);
|
||||||
|
else if (auto * data_uint16 = detail::getIndexesData<UInt16>(indexes))
|
||||||
|
return column.template indexImpl<UInt16>(*data_uint16, limit);
|
||||||
|
else if (auto * data_uint32 = detail::getIndexesData<UInt32>(indexes))
|
||||||
|
return column.template indexImpl<UInt32>(*data_uint32, limit);
|
||||||
|
else if (auto * data_uint64 = detail::getIndexesData<UInt64>(indexes))
|
||||||
|
return column.template indexImpl<UInt64>(*data_uint64, limit);
|
||||||
|
else
|
||||||
|
throw Exception("Indexes column for IColumn::select must be ColumnUInt, got" + indexes.getName(),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
#define INSTANTIATE_INDEX_IMPL(Column) \
|
||||||
|
template ColumnPtr Column::indexImpl<UInt8>(const PaddedPODArray<UInt8> & indexes, size_t limit) const; \
|
||||||
|
template ColumnPtr Column::indexImpl<UInt16>(const PaddedPODArray<UInt16> & indexes, size_t limit) const; \
|
||||||
|
template ColumnPtr Column::indexImpl<UInt32>(const PaddedPODArray<UInt32> & indexes, size_t limit) const; \
|
||||||
|
template ColumnPtr Column::indexImpl<UInt64>(const PaddedPODArray<UInt64> & indexes, size_t limit) const;
|
||||||
}
|
}
|
||||||
|
@ -27,14 +27,14 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column)
|
|||||||
if (column.isColumnConst())
|
if (column.isColumnConst())
|
||||||
{
|
{
|
||||||
const ColumnConst & column_const = static_cast<const ColumnConst &>(column);
|
const ColumnConst & column_const = static_cast<const ColumnConst &>(column);
|
||||||
const IColumn & column_nested = column_const.getDataColumn();
|
ColumnPtr column_nested = column_const.getDataColumnPtr()->convertToFullColumnIfWithDictionary();
|
||||||
|
|
||||||
if (!typeid_cast<const ColumnUInt8 *>(&column_nested))
|
if (!typeid_cast<const ColumnUInt8 *>(column_nested.get()))
|
||||||
{
|
{
|
||||||
const ColumnNullable * column_nested_nullable = typeid_cast<const ColumnNullable *>(&column_nested);
|
const ColumnNullable * column_nested_nullable = typeid_cast<const ColumnNullable *>(column_nested.get());
|
||||||
if (!column_nested_nullable || !typeid_cast<const ColumnUInt8 *>(&column_nested_nullable->getNestedColumn()))
|
if (!column_nested_nullable || !typeid_cast<const ColumnUInt8 *>(&column_nested_nullable->getNestedColumn()))
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type " + column_nested.getName() + " of column for constant filter. Must be UInt8 or Nullable(UInt8).",
|
throw Exception("Illegal type " + column_nested->getName() + " of column for constant filter. Must be UInt8 or Nullable(UInt8).",
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
|
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -48,8 +48,13 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
FilterDescription::FilterDescription(const IColumn & column)
|
FilterDescription::FilterDescription(const IColumn & column_)
|
||||||
{
|
{
|
||||||
|
if (column_.withDictionary())
|
||||||
|
data_holder = column_.convertToFullColumnIfWithDictionary();
|
||||||
|
|
||||||
|
const auto & column = data_holder ? *data_holder : column_;
|
||||||
|
|
||||||
if (const ColumnUInt8 * concrete_column = typeid_cast<const ColumnUInt8 *>(&column))
|
if (const ColumnUInt8 * concrete_column = typeid_cast<const ColumnUInt8 *>(&column))
|
||||||
{
|
{
|
||||||
data = &concrete_column->getData();
|
data = &concrete_column->getData();
|
||||||
|
@ -47,6 +47,10 @@ public:
|
|||||||
*/
|
*/
|
||||||
virtual Ptr convertToFullColumnIfConst() const { return {}; }
|
virtual Ptr convertToFullColumnIfConst() const { return {}; }
|
||||||
|
|
||||||
|
/// If column isn't ColumnWithDictionary, return itself.
|
||||||
|
/// If column is ColumnWithDictionary, transforms is to full column.
|
||||||
|
virtual Ptr convertToFullColumnIfWithDictionary() const { return getPtr(); }
|
||||||
|
|
||||||
/// Creates empty column with the same type.
|
/// Creates empty column with the same type.
|
||||||
virtual MutablePtr cloneEmpty() const { return cloneResized(0); }
|
virtual MutablePtr cloneEmpty() const { return cloneResized(0); }
|
||||||
|
|
||||||
@ -188,6 +192,10 @@ public:
|
|||||||
using Permutation = PaddedPODArray<size_t>;
|
using Permutation = PaddedPODArray<size_t>;
|
||||||
virtual Ptr permute(const Permutation & perm, size_t limit) const = 0;
|
virtual Ptr permute(const Permutation & perm, size_t limit) const = 0;
|
||||||
|
|
||||||
|
/// Creates new column with values column[indexes[:limit]]. If limit is 0, all indexes are used.
|
||||||
|
/// Indexes must be one of the ColumnUInt. For default implementation, see selectIndexImpl from ColumnsCommon.h
|
||||||
|
virtual Ptr index(const IColumn & indexes, size_t limit) const = 0;
|
||||||
|
|
||||||
/** Compares (*this)[n] and rhs[m].
|
/** Compares (*this)[n] and rhs[m].
|
||||||
* Returns negative number, 0, or positive number (*this)[n] is less, equal, greater than rhs[m] respectively.
|
* Returns negative number, 0, or positive number (*this)[n] is less, equal, greater than rhs[m] respectively.
|
||||||
* Is used in sortings.
|
* Is used in sortings.
|
||||||
@ -325,6 +333,8 @@ public:
|
|||||||
/// Can be inside ColumnNullable.
|
/// Can be inside ColumnNullable.
|
||||||
virtual bool canBeInsideNullable() const { return false; }
|
virtual bool canBeInsideNullable() const { return false; }
|
||||||
|
|
||||||
|
virtual bool withDictionary() const { return false; }
|
||||||
|
|
||||||
|
|
||||||
virtual ~IColumn() {}
|
virtual ~IColumn() {}
|
||||||
|
|
||||||
|
@ -87,6 +87,14 @@ public:
|
|||||||
return cloneDummy(limit ? std::min(s, limit) : s);
|
return cloneDummy(limit ? std::min(s, limit) : s);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ColumnPtr index(const IColumn & indexes, size_t limit) const override
|
||||||
|
{
|
||||||
|
if (indexes.size() < limit)
|
||||||
|
throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
return cloneDummy(limit ? limit : s);
|
||||||
|
}
|
||||||
|
|
||||||
void getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const override
|
void getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const override
|
||||||
{
|
{
|
||||||
res.resize(s);
|
res.resize(s);
|
||||||
|
129
dbms/src/Columns/IColumnUnique.h
Normal file
129
dbms/src/Columns/IColumnUnique.h
Normal file
@ -0,0 +1,129 @@
|
|||||||
|
#pragma once
|
||||||
|
#include <Columns/IColumn.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class IColumnUnique : public IColumn
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
using ColumnUniquePtr = IColumn::template immutable_ptr<IColumnUnique>;
|
||||||
|
using MutableColumnUniquePtr = IColumn::template mutable_ptr<IColumnUnique>;
|
||||||
|
|
||||||
|
/// Column always contains Null if it's Nullable and empty string if it's String or Nullable(String).
|
||||||
|
/// So, size may be greater than the number of inserted unique values.
|
||||||
|
virtual const ColumnPtr & getNestedColumn() const = 0;
|
||||||
|
/// The same as getNestedColumn, but removes null map if nested column is nullable.
|
||||||
|
virtual const ColumnPtr & getNestedNotNullableColumn() const = 0;
|
||||||
|
|
||||||
|
size_t size() const override { return getNestedColumn()->size(); }
|
||||||
|
|
||||||
|
/// Appends new value at the end of column (column's size is increased by 1).
|
||||||
|
/// Is used to transform raw strings to Blocks (for example, inside input format parsers)
|
||||||
|
virtual size_t uniqueInsert(const Field & x) = 0;
|
||||||
|
|
||||||
|
virtual size_t uniqueInsertFrom(const IColumn & src, size_t n) = 0;
|
||||||
|
/// Appends range of elements from other column.
|
||||||
|
/// Could be used to concatenate columns.
|
||||||
|
virtual MutableColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) = 0;
|
||||||
|
|
||||||
|
struct IndexesWithOverflow
|
||||||
|
{
|
||||||
|
MutableColumnPtr indexes;
|
||||||
|
MutableColumnPtr overflowed_keys;
|
||||||
|
};
|
||||||
|
/// Like uniqueInsertRangeFrom, but doesn't insert keys if inner dictionary has more than max_dictionary_size keys.
|
||||||
|
/// Keys that won't be inserted into dictionary will be into overflowed_keys, indexes will be calculated for
|
||||||
|
/// concatenation of nested column (which can be got from getNestedColumn() function) and overflowed_keys.
|
||||||
|
virtual IndexesWithOverflow uniqueInsertRangeWithOverflow(const IColumn & src, size_t start,
|
||||||
|
size_t length, size_t max_dictionary_size) = 0;
|
||||||
|
|
||||||
|
/// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented).
|
||||||
|
/// Is used to optimize some computations (in aggregation, for example).
|
||||||
|
/// Parameter length could be ignored if column values have fixed size.
|
||||||
|
virtual size_t uniqueInsertData(const char * pos, size_t length) = 0;
|
||||||
|
virtual size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) = 0;
|
||||||
|
|
||||||
|
virtual size_t getDefaultValueIndex() const = 0;
|
||||||
|
virtual size_t getNullValueIndex() const = 0;
|
||||||
|
virtual bool canContainNulls() const = 0;
|
||||||
|
|
||||||
|
virtual size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) = 0;
|
||||||
|
|
||||||
|
const char * getFamilyName() const override { return "ColumnUnique"; }
|
||||||
|
|
||||||
|
void insert(const Field &) override
|
||||||
|
{
|
||||||
|
throw Exception("Method insert is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
void insertRangeFrom(const IColumn &, size_t, size_t) override
|
||||||
|
{
|
||||||
|
throw Exception("Method insertRangeFrom is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
void insertData(const char *, size_t) override
|
||||||
|
{
|
||||||
|
throw Exception("Method insertData is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
void insertDefault() override
|
||||||
|
{
|
||||||
|
throw Exception("Method insertDefault is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
void popBack(size_t) override
|
||||||
|
{
|
||||||
|
throw Exception("Method popBack is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
void gather(ColumnGathererStream &) override
|
||||||
|
{
|
||||||
|
throw Exception("Method gather is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
const char * deserializeAndInsertFromArena(const char *) override
|
||||||
|
{
|
||||||
|
throw Exception("Method deserializeAndInsertFromArena is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr index(const IColumn &, size_t) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method index is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr cut(size_t, size_t) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method cut is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr filter(const IColumn::Filter &, ssize_t) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method filter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr permute(const IColumn::Permutation &, size_t) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method permute is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr replicate(const IColumn::Offsets &) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method replicate is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
void getPermutation(bool, size_t, int, IColumn::Permutation &) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method getPermutation is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<MutableColumnPtr> scatter(IColumn::ColumnIndex, const IColumn::Selector &) const override
|
||||||
|
{
|
||||||
|
throw Exception("Method scatter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
using ColumnUniquePtr = IColumnUnique::ColumnUniquePtr;
|
||||||
|
using MutableColumnUniquePtr = IColumnUnique::MutableColumnUniquePtr;
|
||||||
|
|
||||||
|
}
|
412
dbms/src/Columns/ReverseIndex.h
Normal file
412
dbms/src/Columns/ReverseIndex.h
Normal file
@ -0,0 +1,412 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Common/HashTable/Hash.h>
|
||||||
|
#include <Common/HashTable/HashTable.h>
|
||||||
|
#include <Common/HashTable/HashTableAllocator.h>
|
||||||
|
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <ext/range.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
template <typename ColumnType, bool with_saved_hash, bool has_base_index>
|
||||||
|
struct ReverseIndexHashTableState;
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ false>
|
||||||
|
{
|
||||||
|
constexpr static bool with_saved_hash = false;
|
||||||
|
constexpr static bool has_base_index = false;
|
||||||
|
|
||||||
|
ColumnType * index_column;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ true>
|
||||||
|
{
|
||||||
|
constexpr static bool with_saved_hash = false;
|
||||||
|
constexpr static bool has_base_index = true;
|
||||||
|
|
||||||
|
ColumnType * index_column;
|
||||||
|
size_t base_index;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ false>
|
||||||
|
{
|
||||||
|
constexpr static bool with_saved_hash = true;
|
||||||
|
constexpr static bool has_base_index = false;
|
||||||
|
|
||||||
|
ColumnType * index_column;
|
||||||
|
typename ColumnVector<UInt64>::Container * saved_hash_column;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename ColumnType>
|
||||||
|
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ true>
|
||||||
|
{
|
||||||
|
constexpr static bool with_saved_hash = true;
|
||||||
|
constexpr static bool has_base_index = true;
|
||||||
|
|
||||||
|
ColumnType * index_column;
|
||||||
|
typename ColumnVector<UInt64>::Container * saved_hash_column;
|
||||||
|
size_t base_index;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
template <typename Hash>
|
||||||
|
struct ReverseIndexHash : public Hash
|
||||||
|
{
|
||||||
|
template <typename T>
|
||||||
|
size_t operator()(T) const
|
||||||
|
{
|
||||||
|
throw Exception("operator()(key) is not implemented for ReverseIndexHash.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename State, typename T>
|
||||||
|
size_t operator()(const State & state, T key) const
|
||||||
|
{
|
||||||
|
auto index = key;
|
||||||
|
if constexpr (State::has_base_index)
|
||||||
|
index -= state.base_index;
|
||||||
|
|
||||||
|
return Hash::operator()(state.index_column->getElement(index));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
using ReverseIndexStringHash = ReverseIndexHash<StringRefHash>;
|
||||||
|
|
||||||
|
template <typename IndexType>
|
||||||
|
using ReverseIndexNumberHash = ReverseIndexHash<DefaultHash<IndexType>>;
|
||||||
|
|
||||||
|
|
||||||
|
template <typename IndexType, typename Hash, typename HashTable, typename ColumnType, bool string_hash, bool has_base_index>
|
||||||
|
struct ReverseIndexHashTableCell
|
||||||
|
: public HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>
|
||||||
|
{
|
||||||
|
using Base = HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>;
|
||||||
|
using State = typename Base::State;
|
||||||
|
using Base::Base;
|
||||||
|
using Base::key;
|
||||||
|
using Base::keyEquals;
|
||||||
|
using Base::isZero;
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
static bool isZero(const T &, const State & /*state*/)
|
||||||
|
{
|
||||||
|
static_assert(!std::is_same_v<typename std::decay<T>::type, typename std::decay<IndexType>::type>);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
/// Special case when we want to compare with something not in index_column.
|
||||||
|
/// When we compare something inside column default keyEquals checks only that row numbers are equal.
|
||||||
|
bool keyEquals(const StringRef & object, size_t hash_ [[maybe_unused]], const State & state) const
|
||||||
|
{
|
||||||
|
auto index = key;
|
||||||
|
if constexpr (has_base_index)
|
||||||
|
index -= state.base_index;
|
||||||
|
|
||||||
|
if constexpr (string_hash)
|
||||||
|
return hash_ == (*state.saved_hash_column)[index] && object == state.index_column->getDataAt(index);
|
||||||
|
else
|
||||||
|
return object == state.index_column->getDataAt(index);
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t getHash(const Hash & hash) const
|
||||||
|
{
|
||||||
|
auto index = key;
|
||||||
|
|
||||||
|
/// Hack. HashTable is Hash itself.
|
||||||
|
const auto & state = static_cast<const State &>(static_cast<const HashTable &>(hash));
|
||||||
|
|
||||||
|
if constexpr (has_base_index)
|
||||||
|
index -= state.base_index;
|
||||||
|
|
||||||
|
if constexpr (string_hash)
|
||||||
|
return (*state.saved_hash_column)[index];
|
||||||
|
else
|
||||||
|
return hash(state, key);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
template <typename Key, typename Cell, typename Hash>
|
||||||
|
class HashTableWithPublicState : public HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>
|
||||||
|
{
|
||||||
|
using State = typename Cell::State;
|
||||||
|
using Base = HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>;
|
||||||
|
|
||||||
|
public:
|
||||||
|
using Base::Base;
|
||||||
|
State & getState() { return *this; }
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType, bool has_base_index>
|
||||||
|
class ReverseIndexStringHashTable : public HashTableWithPublicState<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexHashTableCell<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexStringHash,
|
||||||
|
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
|
||||||
|
ColumnType,
|
||||||
|
true,
|
||||||
|
has_base_index>,
|
||||||
|
ReverseIndexStringHash>
|
||||||
|
{
|
||||||
|
using Base = HashTableWithPublicState<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexHashTableCell<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexStringHash,
|
||||||
|
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
|
||||||
|
ColumnType,
|
||||||
|
true,
|
||||||
|
has_base_index>,
|
||||||
|
ReverseIndexStringHash>;
|
||||||
|
public:
|
||||||
|
using Base::Base;
|
||||||
|
friend struct ReverseIndexHashTableCell<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexStringHash,
|
||||||
|
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
|
||||||
|
ColumnType,
|
||||||
|
true,
|
||||||
|
has_base_index>;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType, bool has_base_index>
|
||||||
|
class ReverseIndexNumberHashTable : public HashTableWithPublicState<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexHashTableCell<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexNumberHash<typename ColumnType::value_type>,
|
||||||
|
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
|
||||||
|
ColumnType,
|
||||||
|
false,
|
||||||
|
has_base_index>,
|
||||||
|
ReverseIndexNumberHash<typename ColumnType::value_type>>
|
||||||
|
{
|
||||||
|
using Base = HashTableWithPublicState<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexHashTableCell<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexNumberHash<typename ColumnType::value_type>,
|
||||||
|
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
|
||||||
|
ColumnType,
|
||||||
|
false,
|
||||||
|
has_base_index>,
|
||||||
|
ReverseIndexNumberHash<typename ColumnType::value_type>>;
|
||||||
|
public:
|
||||||
|
using Base::Base;
|
||||||
|
friend struct ReverseIndexHashTableCell<
|
||||||
|
IndexType,
|
||||||
|
ReverseIndexNumberHash<typename ColumnType::value_type>,
|
||||||
|
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
|
||||||
|
ColumnType,
|
||||||
|
false,
|
||||||
|
has_base_index>;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType, bool has_base_index, bool is_numeric_column>
|
||||||
|
struct SelectReverseIndexHashTable;
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType, bool has_base_index>
|
||||||
|
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, true>
|
||||||
|
{
|
||||||
|
using Type = ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType, bool has_base_index>
|
||||||
|
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, false>
|
||||||
|
{
|
||||||
|
using Type = ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
constexpr bool isNumericColumn(const T *) { return false; }
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
constexpr bool isNumericColumn(const ColumnVector<T> *) { return true; }
|
||||||
|
|
||||||
|
static_assert(isNumericColumn(static_cast<ColumnVector<UInt8> *>(nullptr)));
|
||||||
|
static_assert(!isNumericColumn(static_cast<ColumnString *>(nullptr)));
|
||||||
|
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType, bool has_base_index>
|
||||||
|
using ReverseIndexHashTable = typename SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index,
|
||||||
|
isNumericColumn(static_cast<ColumnType *>(nullptr))>::Type;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType>
|
||||||
|
class ReverseIndex
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit ReverseIndex(UInt64 num_prefix_rows_to_skip, UInt64 base_index)
|
||||||
|
: num_prefix_rows_to_skip(num_prefix_rows_to_skip), base_index(base_index) {}
|
||||||
|
|
||||||
|
void setColumn(ColumnType * column_);
|
||||||
|
|
||||||
|
static constexpr bool is_numeric_column = isNumericColumn(static_cast<ColumnType *>(nullptr));
|
||||||
|
static constexpr bool use_saved_hash = !is_numeric_column;
|
||||||
|
|
||||||
|
UInt64 insert(UInt64 from_position); /// Insert into index column[from_position];
|
||||||
|
UInt64 insertFromLastRow();
|
||||||
|
UInt64 getInsertionPoint(const StringRef & data);
|
||||||
|
UInt64 lastInsertionPoint() const { return size() + base_index; }
|
||||||
|
|
||||||
|
ColumnType * getColumn() const { return column; }
|
||||||
|
size_t size() const;
|
||||||
|
|
||||||
|
size_t allocatedBytes() const { return index ? index->getBufferSizeInBytes() : 0; }
|
||||||
|
|
||||||
|
private:
|
||||||
|
ColumnType * column = nullptr;
|
||||||
|
UInt64 num_prefix_rows_to_skip; /// The number prefix tows in column which won't be sored at index.
|
||||||
|
UInt64 base_index; /// This values will be added to row number which is inserted into index.
|
||||||
|
|
||||||
|
using IndexMapType = ReverseIndexHashTable<IndexType, ColumnType, true>;
|
||||||
|
|
||||||
|
/// Lazy initialized.
|
||||||
|
std::unique_ptr<IndexMapType> index;
|
||||||
|
ColumnUInt64::MutablePtr saved_hash;
|
||||||
|
|
||||||
|
void buildIndex();
|
||||||
|
|
||||||
|
UInt64 getHash(const StringRef & ref) const
|
||||||
|
{
|
||||||
|
if constexpr (is_numeric_column)
|
||||||
|
{
|
||||||
|
using ValueType = typename ColumnType::value_type;
|
||||||
|
ValueType value = *reinterpret_cast<const ValueType *>(ref.data);
|
||||||
|
return DefaultHash<ValueType>()(value);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
return StringRefHash()(ref);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType>
|
||||||
|
void ReverseIndex<IndexType, ColumnType>:: setColumn(ColumnType * column_)
|
||||||
|
{
|
||||||
|
if (column != column_)
|
||||||
|
index = nullptr;
|
||||||
|
|
||||||
|
column = column_;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType>
|
||||||
|
size_t ReverseIndex<IndexType, ColumnType>::size() const
|
||||||
|
{
|
||||||
|
if (!column)
|
||||||
|
throw Exception("ReverseIndex has not size because index column wasn't set.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
return column->size();
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType>
|
||||||
|
void ReverseIndex<IndexType, ColumnType>::buildIndex()
|
||||||
|
{
|
||||||
|
if (index)
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (!column)
|
||||||
|
throw Exception("ReverseIndex can't build index because index column wasn't set.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto size = column->size();
|
||||||
|
index = std::make_unique<IndexMapType>(size);
|
||||||
|
|
||||||
|
if constexpr (use_saved_hash)
|
||||||
|
saved_hash = ColumnUInt64::create(size);
|
||||||
|
|
||||||
|
auto & state = index->getState();
|
||||||
|
state.index_column = column;
|
||||||
|
state.base_index = base_index;
|
||||||
|
if constexpr (use_saved_hash)
|
||||||
|
state.saved_hash_column = &saved_hash->getData();
|
||||||
|
|
||||||
|
using IteratorType = typename IndexMapType::iterator;
|
||||||
|
IteratorType iterator;
|
||||||
|
bool inserted;
|
||||||
|
|
||||||
|
for (auto row : ext::range(num_prefix_rows_to_skip, size))
|
||||||
|
{
|
||||||
|
auto hash = getHash(column->getDataAt(row));
|
||||||
|
|
||||||
|
if constexpr (use_saved_hash)
|
||||||
|
saved_hash->getElement(row) = hash;
|
||||||
|
|
||||||
|
index->emplace(row + base_index, iterator, inserted, hash);
|
||||||
|
|
||||||
|
if (!inserted)
|
||||||
|
throw Exception("Duplicating keys found in ReverseIndex.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType>
|
||||||
|
UInt64 ReverseIndex<IndexType, ColumnType>::insert(UInt64 from_position)
|
||||||
|
{
|
||||||
|
if (!index)
|
||||||
|
buildIndex();
|
||||||
|
|
||||||
|
using IteratorType = typename IndexMapType::iterator;
|
||||||
|
IteratorType iterator;
|
||||||
|
bool inserted;
|
||||||
|
|
||||||
|
auto hash = getHash(column->getDataAt(from_position));
|
||||||
|
|
||||||
|
if constexpr (use_saved_hash)
|
||||||
|
{
|
||||||
|
auto & data = saved_hash->getData();
|
||||||
|
if (data.size() <= from_position)
|
||||||
|
data.resize(from_position + 1);
|
||||||
|
data[from_position] = hash;
|
||||||
|
}
|
||||||
|
|
||||||
|
index->emplace(from_position + base_index, iterator, inserted, hash);
|
||||||
|
|
||||||
|
return *iterator;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType>
|
||||||
|
UInt64 ReverseIndex<IndexType, ColumnType>::insertFromLastRow()
|
||||||
|
{
|
||||||
|
if (!column)
|
||||||
|
throw Exception("ReverseIndex can't insert row from column because index column wasn't set.",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
UInt64 num_rows = size();
|
||||||
|
|
||||||
|
if (num_rows == 0)
|
||||||
|
throw Exception("ReverseIndex can't insert row from column because it is empty.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
UInt64 position = num_rows - 1;
|
||||||
|
UInt64 inserted_pos = insert(position);
|
||||||
|
if (position + base_index != inserted_pos)
|
||||||
|
throw Exception("Can't insert into reverse index from last row (" + toString(position + base_index)
|
||||||
|
+ ") because the same row is in position " + toString(inserted_pos), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
return inserted_pos;
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename IndexType, typename ColumnType>
|
||||||
|
UInt64 ReverseIndex<IndexType, ColumnType>::getInsertionPoint(const StringRef & data)
|
||||||
|
{
|
||||||
|
if (!index)
|
||||||
|
buildIndex();
|
||||||
|
|
||||||
|
using IteratorType = typename IndexMapType::iterator;
|
||||||
|
IteratorType iterator;
|
||||||
|
|
||||||
|
auto hash = getHash(data);
|
||||||
|
iterator = index->find(data, hash);
|
||||||
|
|
||||||
|
return iterator == index->end() ? size() + base_index : *iterator;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -380,9 +380,10 @@ namespace ErrorCodes
|
|||||||
extern const int INVALID_JOIN_ON_EXPRESSION = 403;
|
extern const int INVALID_JOIN_ON_EXPRESSION = 403;
|
||||||
extern const int BAD_ODBC_CONNECTION_STRING = 404;
|
extern const int BAD_ODBC_CONNECTION_STRING = 404;
|
||||||
extern const int PARTITION_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT = 405;
|
extern const int PARTITION_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT = 405;
|
||||||
extern const int BAD_REQUEST_PARAMETER = 406;
|
extern const int TOP_AND_LIMIT_TOGETHER = 406;
|
||||||
extern const int EXTERNAL_EXECUTABLE_NOT_FOUND = 407;
|
extern const int BAD_REQUEST_PARAMETER = 407;
|
||||||
extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING = 408;
|
extern const int EXTERNAL_EXECUTABLE_NOT_FOUND = 408;
|
||||||
|
extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING = 409;
|
||||||
|
|
||||||
extern const int KEEPER_EXCEPTION = 999;
|
extern const int KEEPER_EXCEPTION = 999;
|
||||||
extern const int POCO_EXCEPTION = 1000;
|
extern const int POCO_EXCEPTION = 1000;
|
||||||
|
@ -53,6 +53,7 @@ struct HashMapCell
|
|||||||
|
|
||||||
bool keyEquals(const Key & key_) const { return value.first == key_; }
|
bool keyEquals(const Key & key_) const { return value.first == key_; }
|
||||||
bool keyEquals(const Key & key_, size_t /*hash_*/) const { return value.first == key_; }
|
bool keyEquals(const Key & key_, size_t /*hash_*/) const { return value.first == key_; }
|
||||||
|
bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return value.first == key_; }
|
||||||
|
|
||||||
void setHash(size_t /*hash_value*/) {}
|
void setHash(size_t /*hash_value*/) {}
|
||||||
size_t getHash(const Hash & hash) const { return hash(value.first); }
|
size_t getHash(const Hash & hash) const { return hash(value.first); }
|
||||||
@ -112,6 +113,7 @@ struct HashMapCellWithSavedHash : public HashMapCell<Key, TMapped, Hash, TState>
|
|||||||
|
|
||||||
bool keyEquals(const Key & key_) const { return this->value.first == key_; }
|
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_; }
|
bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->value.first == key_; }
|
||||||
|
bool keyEquals(const Key & key_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); }
|
||||||
|
|
||||||
void setHash(size_t hash_value) { saved_hash = hash_value; }
|
void setHash(size_t hash_value) { saved_hash = hash_value; }
|
||||||
size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; }
|
size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; }
|
||||||
|
@ -75,6 +75,7 @@ struct HashSetCellWithSavedHash : public HashTableCell<Key, Hash, TState>
|
|||||||
|
|
||||||
bool keyEquals(const Key & key_) const { return this->key == key_; }
|
bool keyEquals(const Key & key_) const { return this->key == key_; }
|
||||||
bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->key == key_; }
|
bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->key == key_; }
|
||||||
|
bool keyEquals(const Key & key_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); }
|
||||||
|
|
||||||
void setHash(size_t hash_value) { saved_hash = hash_value; }
|
void setHash(size_t hash_value) { saved_hash = hash_value; }
|
||||||
size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; }
|
size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; }
|
||||||
|
@ -108,6 +108,7 @@ struct HashTableCell
|
|||||||
/// Are the keys at the cells equal?
|
/// Are the keys at the cells equal?
|
||||||
bool keyEquals(const Key & key_) const { return key == key_; }
|
bool keyEquals(const Key & key_) const { return key == key_; }
|
||||||
bool keyEquals(const Key & key_, size_t /*hash_*/) const { return key == key_; }
|
bool keyEquals(const Key & key_, size_t /*hash_*/) const { return key == key_; }
|
||||||
|
bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return key == key_; }
|
||||||
|
|
||||||
/// If the cell can remember the value of the hash function, then remember it.
|
/// If the cell can remember the value of the hash function, then remember it.
|
||||||
void setHash(size_t /*hash_value*/) {}
|
void setHash(size_t /*hash_value*/) {}
|
||||||
@ -280,9 +281,10 @@ protected:
|
|||||||
#endif
|
#endif
|
||||||
|
|
||||||
/// Find a cell with the same key or an empty cell, starting from the specified position and further along the collision resolution chain.
|
/// Find a cell with the same key or an empty cell, starting from the specified position and further along the collision resolution chain.
|
||||||
size_t ALWAYS_INLINE findCell(const Key & x, size_t hash_value, size_t place_value) const
|
template <typename ObjectToCompareWith>
|
||||||
|
size_t ALWAYS_INLINE findCell(const ObjectToCompareWith & x, size_t hash_value, size_t place_value) const
|
||||||
{
|
{
|
||||||
while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value))
|
while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value, *this))
|
||||||
{
|
{
|
||||||
place_value = grower.next(place_value);
|
place_value = grower.next(place_value);
|
||||||
#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
|
#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
|
||||||
@ -734,7 +736,8 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
iterator ALWAYS_INLINE find(Key x)
|
template <typename ObjectToCompareWith>
|
||||||
|
iterator ALWAYS_INLINE find(ObjectToCompareWith x)
|
||||||
{
|
{
|
||||||
if (Cell::isZero(x, *this))
|
if (Cell::isZero(x, *this))
|
||||||
return this->hasZero() ? iteratorToZero() : end();
|
return this->hasZero() ? iteratorToZero() : end();
|
||||||
@ -745,7 +748,8 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
const_iterator ALWAYS_INLINE find(Key x) const
|
template <typename ObjectToCompareWith>
|
||||||
|
const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const
|
||||||
{
|
{
|
||||||
if (Cell::isZero(x, *this))
|
if (Cell::isZero(x, *this))
|
||||||
return this->hasZero() ? iteratorToZero() : end();
|
return this->hasZero() ? iteratorToZero() : end();
|
||||||
@ -756,7 +760,8 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
iterator ALWAYS_INLINE find(Key x, size_t hash_value)
|
template <typename ObjectToCompareWith>
|
||||||
|
iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value)
|
||||||
{
|
{
|
||||||
if (Cell::isZero(x, *this))
|
if (Cell::isZero(x, *this))
|
||||||
return this->hasZero() ? iteratorToZero() : end();
|
return this->hasZero() ? iteratorToZero() : end();
|
||||||
@ -766,7 +771,8 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
const_iterator ALWAYS_INLINE find(Key x, size_t hash_value) const
|
template <typename ObjectToCompareWith>
|
||||||
|
const_iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value) const
|
||||||
{
|
{
|
||||||
if (Cell::isZero(x, *this))
|
if (Cell::isZero(x, *this))
|
||||||
return this->hasZero() ? iteratorToZero() : end();
|
return this->hasZero() ? iteratorToZero() : end();
|
||||||
|
@ -368,6 +368,18 @@ Block Block::cloneWithColumns(const Columns & columns) const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Block Block::cloneWithoutColumns() const
|
||||||
|
{
|
||||||
|
Block res;
|
||||||
|
|
||||||
|
size_t num_columns = data.size();
|
||||||
|
for (size_t i = 0; i < num_columns; ++i)
|
||||||
|
res.insert({ nullptr, data[i].type, data[i].name });
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
Block Block::sortColumns() const
|
Block Block::sortColumns() const
|
||||||
{
|
{
|
||||||
Block sorted_block;
|
Block sorted_block;
|
||||||
|
@ -109,6 +109,7 @@ public:
|
|||||||
Columns getColumns() const;
|
Columns getColumns() const;
|
||||||
void setColumns(const Columns & columns);
|
void setColumns(const Columns & columns);
|
||||||
Block cloneWithColumns(const Columns & columns) const;
|
Block cloneWithColumns(const Columns & columns) const;
|
||||||
|
Block cloneWithoutColumns() const;
|
||||||
|
|
||||||
/** Get empty columns with the same types as in block. */
|
/** Get empty columns with the same types as in block. */
|
||||||
MutableColumns cloneEmptyColumns() const;
|
MutableColumns cloneEmptyColumns() const;
|
||||||
|
@ -0,0 +1,57 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DataStreams/IProfilingBlockInputStream.h>
|
||||||
|
#include <Columns/ColumnWithDictionary.h>
|
||||||
|
#include <DataTypes/DataTypeWithDictionary.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
/** Combines several sources into one.
|
||||||
|
* Unlike UnionBlockInputStream, it does this sequentially.
|
||||||
|
* Blocks of different sources are not interleaved with each other.
|
||||||
|
*/
|
||||||
|
class ConvertColumnWithDictionaryToFullBlockInputStream : public IProfilingBlockInputStream
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit ConvertColumnWithDictionaryToFullBlockInputStream(const BlockInputStreamPtr & input)
|
||||||
|
{
|
||||||
|
children.push_back(input);
|
||||||
|
}
|
||||||
|
|
||||||
|
String getName() const override { return "ConvertColumnWithDictionaryToFull"; }
|
||||||
|
|
||||||
|
Block getHeader() const override { return convert(children.at(0)->getHeader()); }
|
||||||
|
|
||||||
|
protected:
|
||||||
|
Block readImpl() override { return convert(children.back()->read()); }
|
||||||
|
|
||||||
|
private:
|
||||||
|
Block convert(Block && block) const
|
||||||
|
{
|
||||||
|
for (auto & column : block)
|
||||||
|
{
|
||||||
|
auto * type_with_dict = typeid_cast<const DataTypeWithDictionary *>(column.type.get());
|
||||||
|
auto * col_with_dict = typeid_cast<const ColumnWithDictionary *>(column.column.get());
|
||||||
|
|
||||||
|
if (type_with_dict && !col_with_dict)
|
||||||
|
throw Exception("Invalid column for " + type_with_dict->getName() + ": " + column.column->getName(),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
if (!type_with_dict && col_with_dict)
|
||||||
|
throw Exception("Invalid type for " + col_with_dict->getName() + ": " + column.type->getName(),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
if (type_with_dict && col_with_dict)
|
||||||
|
{
|
||||||
|
column.column = col_with_dict->convertToFullColumn();
|
||||||
|
column.type = type_with_dict->getDictionaryType();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return std::move(block);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -59,8 +59,14 @@ NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server
|
|||||||
|
|
||||||
void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint)
|
void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint)
|
||||||
{
|
{
|
||||||
IDataType::InputStreamGetter input_stream_getter = [&] (const IDataType::SubstreamPath &) { return &istr; };
|
IDataType::DeserializeBinaryBulkSettings settings;
|
||||||
type.deserializeBinaryBulkWithMultipleStreams(column, input_stream_getter, rows, avg_value_size_hint, false, {});
|
settings.getter = [&](IDataType::SubstreamPath) -> ReadBuffer * { return &istr; };
|
||||||
|
settings.avg_value_size_hint = avg_value_size_hint;
|
||||||
|
settings.position_independent_encoding = false;
|
||||||
|
|
||||||
|
IDataType::DeserializeBinaryBulkStatePtr state;
|
||||||
|
type.deserializeBinaryBulkStatePrefix(settings, state);
|
||||||
|
type.deserializeBinaryBulkWithMultipleStreams(column, rows, settings, state);
|
||||||
|
|
||||||
if (column.size() != rows)
|
if (column.size() != rows)
|
||||||
throw Exception("Cannot read all data in NativeBlockInputStream. Rows read: " + toString(column.size()) + ". Rows expected: " + toString(rows) + ".",
|
throw Exception("Cannot read all data in NativeBlockInputStream. Rows read: " + toString(column.size()) + ". Rows expected: " + toString(rows) + ".",
|
||||||
|
@ -52,8 +52,15 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr
|
|||||||
else
|
else
|
||||||
full_column = column;
|
full_column = column;
|
||||||
|
|
||||||
IDataType::OutputStreamGetter output_stream_getter = [&] (const IDataType::SubstreamPath &) { return &ostr; };
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
type.serializeBinaryBulkWithMultipleStreams(*full_column, output_stream_getter, offset, limit, false, {});
|
settings.getter = [&ostr](IDataType::SubstreamPath) -> WriteBuffer * { return &ostr; };
|
||||||
|
settings.position_independent_encoding = false;
|
||||||
|
settings.low_cardinality_max_dictionary_size = 0;
|
||||||
|
|
||||||
|
IDataType::SerializeBinaryBulkStatePtr state;
|
||||||
|
type.serializeBinaryBulkStatePrefix(settings, state);
|
||||||
|
type.serializeBinaryBulkWithMultipleStreams(*full_column, offset, limit, settings, state);
|
||||||
|
type.serializeBinaryBulkStateSuffix(settings, state);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -145,37 +145,67 @@ namespace
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DataTypeArray::enumerateStreams(StreamCallback callback, SubstreamPath path) const
|
void DataTypeArray::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
|
||||||
{
|
{
|
||||||
path.push_back(Substream::ArraySizes);
|
path.push_back(Substream::ArraySizes);
|
||||||
callback(path);
|
callback(path);
|
||||||
path.back() = Substream::ArrayElements;
|
path.back() = Substream::ArrayElements;
|
||||||
nested->enumerateStreams(callback, path);
|
nested->enumerateStreams(callback, path);
|
||||||
|
path.pop_back();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void DataTypeArray::serializeBinaryBulkStatePrefix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
settings.path.push_back(Substream::ArrayElements);
|
||||||
|
nested->serializeBinaryBulkStatePrefix(settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void DataTypeArray::serializeBinaryBulkStateSuffix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
settings.path.push_back(Substream::ArrayElements);
|
||||||
|
nested->serializeBinaryBulkStateSuffix(settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void DataTypeArray::deserializeBinaryBulkStatePrefix(
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
settings.path.push_back(Substream::ArrayElements);
|
||||||
|
nested->deserializeBinaryBulkStatePrefix(settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DataTypeArray::serializeBinaryBulkWithMultipleStreams(
|
void DataTypeArray::serializeBinaryBulkWithMultipleStreams(
|
||||||
const IColumn & column,
|
const IColumn & column,
|
||||||
OutputStreamGetter getter,
|
|
||||||
size_t offset,
|
size_t offset,
|
||||||
size_t limit,
|
size_t limit,
|
||||||
bool position_independent_encoding,
|
SerializeBinaryBulkSettings & settings,
|
||||||
SubstreamPath path) const
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
{
|
{
|
||||||
const ColumnArray & column_array = typeid_cast<const ColumnArray &>(column);
|
const ColumnArray & column_array = typeid_cast<const ColumnArray &>(column);
|
||||||
|
|
||||||
/// First serialize array sizes.
|
/// First serialize array sizes.
|
||||||
path.push_back(Substream::ArraySizes);
|
settings.path.push_back(Substream::ArraySizes);
|
||||||
if (auto stream = getter(path))
|
if (auto stream = settings.getter(settings.path))
|
||||||
{
|
{
|
||||||
if (position_independent_encoding)
|
if (settings.position_independent_encoding)
|
||||||
serializeArraySizesPositionIndependent(column, *stream, offset, limit);
|
serializeArraySizesPositionIndependent(column, *stream, offset, limit);
|
||||||
else
|
else
|
||||||
DataTypeNumber<ColumnArray::Offset>().serializeBinaryBulk(*column_array.getOffsetsPtr(), *stream, offset, limit);
|
DataTypeNumber<ColumnArray::Offset>().serializeBinaryBulk(*column_array.getOffsetsPtr(), *stream, offset, limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Then serialize contents of arrays.
|
/// Then serialize contents of arrays.
|
||||||
path.back() = Substream::ArrayElements;
|
settings.path.back() = Substream::ArrayElements;
|
||||||
const ColumnArray::Offsets & offset_values = column_array.getOffsets();
|
const ColumnArray::Offsets & offset_values = column_array.getOffsets();
|
||||||
|
|
||||||
if (offset > offset_values.size())
|
if (offset > offset_values.size())
|
||||||
@ -197,30 +227,29 @@ void DataTypeArray::serializeBinaryBulkWithMultipleStreams(
|
|||||||
: 0;
|
: 0;
|
||||||
|
|
||||||
if (limit == 0 || nested_limit)
|
if (limit == 0 || nested_limit)
|
||||||
nested->serializeBinaryBulkWithMultipleStreams(column_array.getData(), getter, nested_offset, nested_limit, position_independent_encoding, path);
|
nested->serializeBinaryBulkWithMultipleStreams(column_array.getData(), nested_offset, nested_limit, settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DataTypeArray::deserializeBinaryBulkWithMultipleStreams(
|
void DataTypeArray::deserializeBinaryBulkWithMultipleStreams(
|
||||||
IColumn & column,
|
IColumn & column,
|
||||||
InputStreamGetter getter,
|
|
||||||
size_t limit,
|
size_t limit,
|
||||||
double /*avg_value_size_hint*/,
|
DeserializeBinaryBulkSettings & settings,
|
||||||
bool position_independent_encoding,
|
DeserializeBinaryBulkStatePtr & state) const
|
||||||
SubstreamPath path) const
|
|
||||||
{
|
{
|
||||||
ColumnArray & column_array = typeid_cast<ColumnArray &>(column);
|
ColumnArray & column_array = typeid_cast<ColumnArray &>(column);
|
||||||
|
|
||||||
path.push_back(Substream::ArraySizes);
|
settings.path.push_back(Substream::ArraySizes);
|
||||||
if (auto stream = getter(path))
|
if (auto stream = settings.getter(settings.path))
|
||||||
{
|
{
|
||||||
if (position_independent_encoding)
|
if (settings.position_independent_encoding)
|
||||||
deserializeArraySizesPositionIndependent(column, *stream, limit);
|
deserializeArraySizesPositionIndependent(column, *stream, limit);
|
||||||
else
|
else
|
||||||
DataTypeNumber<ColumnArray::Offset>().deserializeBinaryBulk(column_array.getOffsetsColumn(), *stream, limit, 0);
|
DataTypeNumber<ColumnArray::Offset>().deserializeBinaryBulk(column_array.getOffsetsColumn(), *stream, limit, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
path.back() = Substream::ArrayElements;
|
settings.path.back() = Substream::ArrayElements;
|
||||||
|
|
||||||
ColumnArray::Offsets & offset_values = column_array.getOffsets();
|
ColumnArray::Offsets & offset_values = column_array.getOffsets();
|
||||||
IColumn & nested_column = column_array.getData();
|
IColumn & nested_column = column_array.getData();
|
||||||
@ -230,7 +259,8 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreams(
|
|||||||
if (last_offset < nested_column.size())
|
if (last_offset < nested_column.size())
|
||||||
throw Exception("Nested column is longer than last offset", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Nested column is longer than last offset", ErrorCodes::LOGICAL_ERROR);
|
||||||
size_t nested_limit = last_offset - nested_column.size();
|
size_t nested_limit = last_offset - nested_column.size();
|
||||||
nested->deserializeBinaryBulkWithMultipleStreams(nested_column, getter, nested_limit, 0, position_independent_encoding, path);
|
nested->deserializeBinaryBulkWithMultipleStreams(nested_column, nested_limit, settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
|
|
||||||
/// Check consistency between offsets and elements subcolumns.
|
/// Check consistency between offsets and elements subcolumns.
|
||||||
/// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER.
|
/// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER.
|
||||||
|
@ -61,23 +61,32 @@ public:
|
|||||||
* This is necessary, because when implementing nested structures, several arrays can have common sizes.
|
* This is necessary, because when implementing nested structures, several arrays can have common sizes.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
void enumerateStreams(StreamCallback callback, SubstreamPath path) const override;
|
void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override;
|
||||||
|
|
||||||
|
void serializeBinaryBulkStatePrefix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void serializeBinaryBulkStateSuffix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void deserializeBinaryBulkStatePrefix(
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
void serializeBinaryBulkWithMultipleStreams(
|
void serializeBinaryBulkWithMultipleStreams(
|
||||||
const IColumn & column,
|
const IColumn & column,
|
||||||
OutputStreamGetter getter,
|
size_t offset,
|
||||||
size_t offset,
|
size_t limit,
|
||||||
size_t limit,
|
SerializeBinaryBulkSettings & settings,
|
||||||
bool position_independent_encoding,
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
SubstreamPath path) const override;
|
|
||||||
|
|
||||||
void deserializeBinaryBulkWithMultipleStreams(
|
void deserializeBinaryBulkWithMultipleStreams(
|
||||||
IColumn & column,
|
IColumn & column,
|
||||||
InputStreamGetter getter,
|
size_t limit,
|
||||||
size_t limit,
|
DeserializeBinaryBulkSettings & settings,
|
||||||
double avg_value_size_hint,
|
DeserializeBinaryBulkStatePtr & state) const override;
|
||||||
bool position_independent_encoding,
|
|
||||||
SubstreamPath path) const override;
|
|
||||||
|
|
||||||
MutableColumnPtr createColumn() const override;
|
MutableColumnPtr createColumn() const override;
|
||||||
|
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Poco/String.h>
|
#include <Poco/String.h>
|
||||||
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -55,6 +56,23 @@ DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr
|
|||||||
{
|
{
|
||||||
String family_name = getAliasToOrName(family_name_param);
|
String family_name = getAliasToOrName(family_name_param);
|
||||||
|
|
||||||
|
if (endsWith(family_name, "WithDictionary"))
|
||||||
|
{
|
||||||
|
ASTPtr low_cardinality_params = std::make_shared<ASTExpressionList>();
|
||||||
|
String param_name = family_name.substr(0, family_name.size() - strlen("WithDictionary"));
|
||||||
|
if (parameters)
|
||||||
|
{
|
||||||
|
auto func = std::make_shared<ASTFunction>();
|
||||||
|
func->name = param_name;
|
||||||
|
func->arguments = parameters;
|
||||||
|
low_cardinality_params->children.push_back(func);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
low_cardinality_params->children.push_back(std::make_shared<ASTIdentifier>(param_name));
|
||||||
|
|
||||||
|
return get("LowCardinality", low_cardinality_params);
|
||||||
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
DataTypesDictionary::const_iterator it = data_types.find(family_name);
|
DataTypesDictionary::const_iterator it = data_types.find(family_name);
|
||||||
if (data_types.end() != it)
|
if (data_types.end() != it)
|
||||||
@ -124,6 +142,7 @@ void registerDataTypeUUID(DataTypeFactory & factory);
|
|||||||
void registerDataTypeAggregateFunction(DataTypeFactory & factory);
|
void registerDataTypeAggregateFunction(DataTypeFactory & factory);
|
||||||
void registerDataTypeNested(DataTypeFactory & factory);
|
void registerDataTypeNested(DataTypeFactory & factory);
|
||||||
void registerDataTypeInterval(DataTypeFactory & factory);
|
void registerDataTypeInterval(DataTypeFactory & factory);
|
||||||
|
void registerDataTypeWithDictionary(DataTypeFactory & factory);
|
||||||
|
|
||||||
|
|
||||||
DataTypeFactory::DataTypeFactory()
|
DataTypeFactory::DataTypeFactory()
|
||||||
@ -142,6 +161,7 @@ DataTypeFactory::DataTypeFactory()
|
|||||||
registerDataTypeAggregateFunction(*this);
|
registerDataTypeAggregateFunction(*this);
|
||||||
registerDataTypeNested(*this);
|
registerDataTypeNested(*this);
|
||||||
registerDataTypeInterval(*this);
|
registerDataTypeInterval(*this);
|
||||||
|
registerDataTypeWithDictionary(*this);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -37,53 +37,83 @@ bool DataTypeNullable::onlyNull() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DataTypeNullable::enumerateStreams(StreamCallback callback, SubstreamPath path) const
|
void DataTypeNullable::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
|
||||||
{
|
{
|
||||||
path.push_back(Substream::NullMap);
|
path.push_back(Substream::NullMap);
|
||||||
callback(path);
|
callback(path);
|
||||||
path.back() = Substream::NullableElements;
|
path.back() = Substream::NullableElements;
|
||||||
nested_data_type->enumerateStreams(callback, path);
|
nested_data_type->enumerateStreams(callback, path);
|
||||||
|
path.pop_back();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void DataTypeNullable::serializeBinaryBulkStatePrefix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
settings.path.push_back(Substream::NullableElements);
|
||||||
|
nested_data_type->serializeBinaryBulkStatePrefix(settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void DataTypeNullable::serializeBinaryBulkStateSuffix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
settings.path.push_back(Substream::NullableElements);
|
||||||
|
nested_data_type->serializeBinaryBulkStateSuffix(settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void DataTypeNullable::deserializeBinaryBulkStatePrefix(
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
settings.path.push_back(Substream::NullableElements);
|
||||||
|
nested_data_type->deserializeBinaryBulkStatePrefix(settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DataTypeNullable::serializeBinaryBulkWithMultipleStreams(
|
void DataTypeNullable::serializeBinaryBulkWithMultipleStreams(
|
||||||
const IColumn & column,
|
const IColumn & column,
|
||||||
OutputStreamGetter getter,
|
|
||||||
size_t offset,
|
size_t offset,
|
||||||
size_t limit,
|
size_t limit,
|
||||||
bool position_independent_encoding,
|
SerializeBinaryBulkSettings & settings,
|
||||||
SubstreamPath path) const
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
{
|
{
|
||||||
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
|
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
|
||||||
col.checkConsistency();
|
col.checkConsistency();
|
||||||
|
|
||||||
/// First serialize null map.
|
/// First serialize null map.
|
||||||
path.push_back(Substream::NullMap);
|
settings.path.push_back(Substream::NullMap);
|
||||||
if (auto stream = getter(path))
|
if (auto stream = settings.getter(settings.path))
|
||||||
DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), *stream, offset, limit);
|
DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), *stream, offset, limit);
|
||||||
|
|
||||||
/// Then serialize contents of arrays.
|
/// Then serialize contents of arrays.
|
||||||
path.back() = Substream::NullableElements;
|
settings.path.back() = Substream::NullableElements;
|
||||||
nested_data_type->serializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, offset, limit, position_independent_encoding, path);
|
nested_data_type->serializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), offset, limit, settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams(
|
void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams(
|
||||||
IColumn & column,
|
IColumn & column,
|
||||||
InputStreamGetter getter,
|
|
||||||
size_t limit,
|
size_t limit,
|
||||||
double avg_value_size_hint,
|
DeserializeBinaryBulkSettings & settings,
|
||||||
bool position_independent_encoding,
|
DeserializeBinaryBulkStatePtr & state) const
|
||||||
SubstreamPath path) const
|
|
||||||
{
|
{
|
||||||
ColumnNullable & col = static_cast<ColumnNullable &>(column);
|
ColumnNullable & col = static_cast<ColumnNullable &>(column);
|
||||||
|
|
||||||
path.push_back(Substream::NullMap);
|
settings.path.push_back(Substream::NullMap);
|
||||||
if (auto stream = getter(path))
|
if (auto stream = settings.getter(settings.path))
|
||||||
DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0);
|
DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0);
|
||||||
|
|
||||||
path.back() = Substream::NullableElements;
|
settings.path.back() = Substream::NullableElements;
|
||||||
nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, limit, avg_value_size_hint, position_independent_encoding, path);
|
nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), limit, settings, state);
|
||||||
|
settings.path.pop_back();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -13,27 +13,36 @@ class DataTypeNullable final : public IDataType
|
|||||||
public:
|
public:
|
||||||
static constexpr bool is_parametric = true;
|
static constexpr bool is_parametric = true;
|
||||||
|
|
||||||
DataTypeNullable(const DataTypePtr & nested_data_type_);
|
explicit DataTypeNullable(const DataTypePtr & nested_data_type_);
|
||||||
std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
|
std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
|
||||||
const char * getFamilyName() const override { return "Nullable"; }
|
const char * getFamilyName() const override { return "Nullable"; }
|
||||||
|
|
||||||
void enumerateStreams(StreamCallback callback, SubstreamPath path) const override;
|
void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override;
|
||||||
|
|
||||||
|
void serializeBinaryBulkStatePrefix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void serializeBinaryBulkStateSuffix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void deserializeBinaryBulkStatePrefix(
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
void serializeBinaryBulkWithMultipleStreams(
|
void serializeBinaryBulkWithMultipleStreams(
|
||||||
const IColumn & column,
|
const IColumn & column,
|
||||||
OutputStreamGetter getter,
|
size_t offset,
|
||||||
size_t offset,
|
size_t limit,
|
||||||
size_t limit,
|
SerializeBinaryBulkSettings & settings,
|
||||||
bool position_independent_encoding,
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
SubstreamPath path) const override;
|
|
||||||
|
|
||||||
void deserializeBinaryBulkWithMultipleStreams(
|
void deserializeBinaryBulkWithMultipleStreams(
|
||||||
IColumn & column,
|
IColumn & column,
|
||||||
InputStreamGetter getter,
|
size_t limit,
|
||||||
size_t limit,
|
DeserializeBinaryBulkSettings & settings,
|
||||||
double avg_value_size_hint,
|
DeserializeBinaryBulkStatePtr & state) const override;
|
||||||
bool position_independent_encoding,
|
|
||||||
SubstreamPath path) const override;
|
|
||||||
|
|
||||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const override { nested_data_type->serializeBinary(field, ostr); }
|
void serializeBinary(const Field & field, WriteBuffer & ostr) const override { nested_data_type->serializeBinary(field, ostr); }
|
||||||
void deserializeBinary(Field & field, ReadBuffer & istr) const override { nested_data_type->deserializeBinary(field, istr); }
|
void deserializeBinary(Field & field, ReadBuffer & istr) const override { nested_data_type->deserializeBinary(field, istr); }
|
||||||
|
@ -283,7 +283,7 @@ void DataTypeTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
void DataTypeTuple::enumerateStreams(StreamCallback callback, SubstreamPath path) const
|
void DataTypeTuple::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
|
||||||
{
|
{
|
||||||
path.push_back(Substream::TupleElement);
|
path.push_back(Substream::TupleElement);
|
||||||
for (const auto i : ext::range(0, ext::size(elems)))
|
for (const auto i : ext::range(0, ext::size(elems)))
|
||||||
@ -291,40 +291,139 @@ void DataTypeTuple::enumerateStreams(StreamCallback callback, SubstreamPath path
|
|||||||
path.back().tuple_element_name = names[i];
|
path.back().tuple_element_name = names[i];
|
||||||
elems[i]->enumerateStreams(callback, path);
|
elems[i]->enumerateStreams(callback, path);
|
||||||
}
|
}
|
||||||
|
path.pop_back();
|
||||||
|
}
|
||||||
|
|
||||||
|
struct SerializeBinaryBulkStateTuple : public IDataType::SerializeBinaryBulkState
|
||||||
|
{
|
||||||
|
std::vector<IDataType::SerializeBinaryBulkStatePtr> states;
|
||||||
|
};
|
||||||
|
|
||||||
|
struct DeserializeBinaryBulkStateTuple : public IDataType::DeserializeBinaryBulkState
|
||||||
|
{
|
||||||
|
std::vector<IDataType::DeserializeBinaryBulkStatePtr> states;
|
||||||
|
};
|
||||||
|
|
||||||
|
static SerializeBinaryBulkStateTuple * checkAndGetTupleSerializeState(IDataType::SerializeBinaryBulkStatePtr & state)
|
||||||
|
{
|
||||||
|
if (!state)
|
||||||
|
throw Exception("Got empty state for DataTypeTuple.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto * tuple_state = typeid_cast<SerializeBinaryBulkStateTuple *>(state.get());
|
||||||
|
if (!tuple_state)
|
||||||
|
{
|
||||||
|
auto & state_ref = *state;
|
||||||
|
throw Exception("Invalid SerializeBinaryBulkState for DataTypeTuple. Expected: "
|
||||||
|
+ demangle(typeid(SerializeBinaryBulkStateTuple).name()) + ", got "
|
||||||
|
+ demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
return tuple_state;
|
||||||
|
}
|
||||||
|
|
||||||
|
static DeserializeBinaryBulkStateTuple * checkAndGetTupleDeserializeState(IDataType::DeserializeBinaryBulkStatePtr & state)
|
||||||
|
{
|
||||||
|
if (!state)
|
||||||
|
throw Exception("Got empty state for DataTypeTuple.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto * tuple_state = typeid_cast<DeserializeBinaryBulkStateTuple *>(state.get());
|
||||||
|
if (!tuple_state)
|
||||||
|
{
|
||||||
|
auto & state_ref = *state;
|
||||||
|
throw Exception("Invalid DeserializeBinaryBulkState for DataTypeTuple. Expected: "
|
||||||
|
+ demangle(typeid(DeserializeBinaryBulkStateTuple).name()) + ", got "
|
||||||
|
+ demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
return tuple_state;
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeTuple::serializeBinaryBulkStatePrefix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
auto tuple_state = std::make_shared<SerializeBinaryBulkStateTuple>();
|
||||||
|
tuple_state->states.resize(elems.size());
|
||||||
|
|
||||||
|
settings.path.push_back(Substream::TupleElement);
|
||||||
|
for (size_t i = 0; i < elems.size(); ++i)
|
||||||
|
{
|
||||||
|
settings.path.back().tuple_element_name = names[i];
|
||||||
|
elems[i]->serializeBinaryBulkStatePrefix(settings, tuple_state->states[i]);
|
||||||
|
}
|
||||||
|
settings.path.pop_back();
|
||||||
|
|
||||||
|
state = std::move(tuple_state);
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeTuple::serializeBinaryBulkStateSuffix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
auto * tuple_state = checkAndGetTupleSerializeState(state);
|
||||||
|
|
||||||
|
settings.path.push_back(Substream::TupleElement);
|
||||||
|
for (size_t i = 0; i < elems.size(); ++i)
|
||||||
|
{
|
||||||
|
settings.path.back().tuple_element_name = names[i];
|
||||||
|
elems[i]->serializeBinaryBulkStateSuffix(settings, tuple_state->states[i]);
|
||||||
|
}
|
||||||
|
settings.path.pop_back();
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeTuple::deserializeBinaryBulkStatePrefix(
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
auto tuple_state = std::make_shared<DeserializeBinaryBulkStateTuple>();
|
||||||
|
tuple_state->states.resize(elems.size());
|
||||||
|
|
||||||
|
settings.path.push_back(Substream::TupleElement);
|
||||||
|
for (size_t i = 0; i < elems.size(); ++i)
|
||||||
|
{
|
||||||
|
settings.path.back().tuple_element_name = names[i];
|
||||||
|
elems[i]->deserializeBinaryBulkStatePrefix(settings, tuple_state->states[i]);
|
||||||
|
}
|
||||||
|
settings.path.pop_back();
|
||||||
|
|
||||||
|
state = std::move(tuple_state);
|
||||||
}
|
}
|
||||||
|
|
||||||
void DataTypeTuple::serializeBinaryBulkWithMultipleStreams(
|
void DataTypeTuple::serializeBinaryBulkWithMultipleStreams(
|
||||||
const IColumn & column,
|
const IColumn & column,
|
||||||
OutputStreamGetter getter,
|
|
||||||
size_t offset,
|
size_t offset,
|
||||||
size_t limit,
|
size_t limit,
|
||||||
bool position_independent_encoding,
|
SerializeBinaryBulkSettings & settings,
|
||||||
SubstreamPath path) const
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
{
|
{
|
||||||
path.push_back(Substream::TupleElement);
|
auto * tuple_state = checkAndGetTupleSerializeState(state);
|
||||||
|
|
||||||
|
settings.path.push_back(Substream::TupleElement);
|
||||||
for (const auto i : ext::range(0, ext::size(elems)))
|
for (const auto i : ext::range(0, ext::size(elems)))
|
||||||
{
|
{
|
||||||
path.back().tuple_element_name = names[i];
|
settings.path.back().tuple_element_name = names[i];
|
||||||
elems[i]->serializeBinaryBulkWithMultipleStreams(
|
auto & element_col = extractElementColumn(column, i);
|
||||||
extractElementColumn(column, i), getter, offset, limit, position_independent_encoding, path);
|
elems[i]->serializeBinaryBulkWithMultipleStreams(element_col, offset, limit, settings, tuple_state->states[i]);
|
||||||
}
|
}
|
||||||
|
settings.path.pop_back();
|
||||||
}
|
}
|
||||||
|
|
||||||
void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams(
|
void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams(
|
||||||
IColumn & column,
|
IColumn & column,
|
||||||
InputStreamGetter getter,
|
|
||||||
size_t limit,
|
size_t limit,
|
||||||
double avg_value_size_hint,
|
DeserializeBinaryBulkSettings & settings,
|
||||||
bool position_independent_encoding,
|
DeserializeBinaryBulkStatePtr & state) const
|
||||||
SubstreamPath path) const
|
|
||||||
{
|
{
|
||||||
path.push_back(Substream::TupleElement);
|
auto * tuple_state = checkAndGetTupleDeserializeState(state);
|
||||||
|
|
||||||
|
settings.path.push_back(Substream::TupleElement);
|
||||||
for (const auto i : ext::range(0, ext::size(elems)))
|
for (const auto i : ext::range(0, ext::size(elems)))
|
||||||
{
|
{
|
||||||
path.back().tuple_element_name = names[i];
|
settings.path.back().tuple_element_name = names[i];
|
||||||
elems[i]->deserializeBinaryBulkWithMultipleStreams(
|
auto & element_col = extractElementColumn(column, i);
|
||||||
extractElementColumn(column, i), getter, limit, avg_value_size_hint, position_independent_encoding, path);
|
elems[i]->deserializeBinaryBulkWithMultipleStreams(element_col, limit, settings, tuple_state->states[i]);
|
||||||
}
|
}
|
||||||
|
settings.path.pop_back();
|
||||||
}
|
}
|
||||||
|
|
||||||
MutableColumnPtr DataTypeTuple::createColumn() const
|
MutableColumnPtr DataTypeTuple::createColumn() const
|
||||||
|
@ -53,23 +53,32 @@ public:
|
|||||||
|
|
||||||
/** Each sub-column in a tuple is serialized in separate stream.
|
/** Each sub-column in a tuple is serialized in separate stream.
|
||||||
*/
|
*/
|
||||||
void enumerateStreams(StreamCallback callback, SubstreamPath path) const override;
|
void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override;
|
||||||
|
|
||||||
|
void serializeBinaryBulkStatePrefix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void serializeBinaryBulkStateSuffix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void deserializeBinaryBulkStatePrefix(
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
void serializeBinaryBulkWithMultipleStreams(
|
void serializeBinaryBulkWithMultipleStreams(
|
||||||
const IColumn & column,
|
const IColumn & column,
|
||||||
OutputStreamGetter getter,
|
size_t offset,
|
||||||
size_t offset,
|
size_t limit,
|
||||||
size_t limit,
|
SerializeBinaryBulkSettings & settings,
|
||||||
bool position_independent_encoding,
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
SubstreamPath path) const override;
|
|
||||||
|
|
||||||
void deserializeBinaryBulkWithMultipleStreams(
|
void deserializeBinaryBulkWithMultipleStreams(
|
||||||
IColumn & column,
|
IColumn & column,
|
||||||
InputStreamGetter getter,
|
size_t limit,
|
||||||
size_t limit,
|
DeserializeBinaryBulkSettings & settings,
|
||||||
double avg_value_size_hint,
|
DeserializeBinaryBulkStatePtr & state) const override;
|
||||||
bool position_independent_encoding,
|
|
||||||
SubstreamPath path) const override;
|
|
||||||
|
|
||||||
MutableColumnPtr createColumn() const override;
|
MutableColumnPtr createColumn() const override;
|
||||||
|
|
||||||
|
840
dbms/src/DataTypes/DataTypeWithDictionary.cpp
Normal file
840
dbms/src/DataTypes/DataTypeWithDictionary.cpp
Normal file
@ -0,0 +1,840 @@
|
|||||||
|
#include <Columns/ColumnWithDictionary.h>
|
||||||
|
#include <Columns/ColumnUnique.h>
|
||||||
|
#include <Columns/ColumnFixedString.h>
|
||||||
|
#include <Columns/ColumnsCommon.h>
|
||||||
|
#include <Common/HashTable/HashMap.h>
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Core/TypeListNumber.h>
|
||||||
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
|
#include <DataTypes/DataTypeWithDictionary.h>
|
||||||
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
#include <DataTypes/DataTypeDate.h>
|
||||||
|
#include <DataTypes/DataTypeDateTime.h>
|
||||||
|
#include <Parsers/IAST.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column)
|
||||||
|
{
|
||||||
|
return typeid_cast<const ColumnWithDictionary &>(column);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnWithDictionary & getColumnWithDictionary(IColumn & column)
|
||||||
|
{
|
||||||
|
return typeid_cast<ColumnWithDictionary &>(column);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_)
|
||||||
|
: dictionary_type(std::move(dictionary_type_))
|
||||||
|
{
|
||||||
|
auto inner_type = dictionary_type;
|
||||||
|
if (dictionary_type->isNullable())
|
||||||
|
inner_type = static_cast<const DataTypeNullable &>(*dictionary_type).getNestedType();
|
||||||
|
|
||||||
|
if (!inner_type->isStringOrFixedString()
|
||||||
|
&& !inner_type->isDateOrDateTime()
|
||||||
|
&& !inner_type->isNumber())
|
||||||
|
throw Exception("DataTypeWithDictionary is supported only for numbers, strings, Date or DateTime, but got "
|
||||||
|
+ dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeWithDictionary::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
|
||||||
|
{
|
||||||
|
path.push_back(Substream::DictionaryKeys);
|
||||||
|
dictionary_type->enumerateStreams(callback, path);
|
||||||
|
path.back() = Substream::DictionaryIndexes;
|
||||||
|
callback(path);
|
||||||
|
path.pop_back();
|
||||||
|
}
|
||||||
|
|
||||||
|
struct KeysSerializationVersion
|
||||||
|
{
|
||||||
|
enum Value
|
||||||
|
{
|
||||||
|
/// Version is written at the start of <name.dict.bin>.
|
||||||
|
/// Dictionary is written as number N and N keys after them.
|
||||||
|
/// Dictionary can be shared for continuous range of granules, so some marks may point to the same position.
|
||||||
|
/// Shared dictionary is stored in state and is read once.
|
||||||
|
SharedDictionariesWithAdditionalKeys = 1,
|
||||||
|
};
|
||||||
|
|
||||||
|
Value value;
|
||||||
|
|
||||||
|
static void checkVersion(UInt64 version)
|
||||||
|
{
|
||||||
|
if (version != SharedDictionariesWithAdditionalKeys)
|
||||||
|
throw Exception("Invalid version for DataTypeWithDictionary key column.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
KeysSerializationVersion(UInt64 version) : value(static_cast<Value>(version)) { checkVersion(version); }
|
||||||
|
};
|
||||||
|
|
||||||
|
/// Version is stored at the start of each granule. It's used to store indexes type and flags.
|
||||||
|
struct IndexesSerializationType
|
||||||
|
{
|
||||||
|
using SerializationType = UInt64;
|
||||||
|
/// Need to read dictionary if it wasn't.
|
||||||
|
static constexpr SerializationType NeedGlobalDictionaryBit = 1u << 8u;
|
||||||
|
/// Need to read additional keys. Additional keys are stored before indexes as value N and N keys after them.
|
||||||
|
static constexpr SerializationType HasAdditionalKeysBit = 1u << 9u;
|
||||||
|
/// Need to update dictionary. It means that previous granule has different dictionary.
|
||||||
|
static constexpr SerializationType NeedUpdateDictionary = 1u << 10u;
|
||||||
|
|
||||||
|
enum Type
|
||||||
|
{
|
||||||
|
TUInt8 = 0,
|
||||||
|
TUInt16,
|
||||||
|
TUInt32,
|
||||||
|
TUInt64,
|
||||||
|
};
|
||||||
|
|
||||||
|
Type type;
|
||||||
|
bool has_additional_keys;
|
||||||
|
bool need_global_dictionary;
|
||||||
|
bool need_update_dictionary;
|
||||||
|
|
||||||
|
static constexpr SerializationType resetFlags(SerializationType type)
|
||||||
|
{
|
||||||
|
return type & (~(HasAdditionalKeysBit | NeedGlobalDictionaryBit | NeedUpdateDictionary));
|
||||||
|
}
|
||||||
|
|
||||||
|
static void checkType(SerializationType type)
|
||||||
|
{
|
||||||
|
UInt64 value = resetFlags(type);
|
||||||
|
if (value <= TUInt64)
|
||||||
|
return;
|
||||||
|
|
||||||
|
throw Exception("Invalid type for DataTypeWithDictionary index column.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
void serialize(WriteBuffer & buffer) const
|
||||||
|
{
|
||||||
|
SerializationType val = type;
|
||||||
|
if (has_additional_keys)
|
||||||
|
val |= HasAdditionalKeysBit;
|
||||||
|
if (need_global_dictionary)
|
||||||
|
val |= NeedGlobalDictionaryBit;
|
||||||
|
if (need_update_dictionary)
|
||||||
|
val |= NeedUpdateDictionary;
|
||||||
|
writeIntBinary(val, buffer);
|
||||||
|
}
|
||||||
|
|
||||||
|
void deserialize(ReadBuffer & buffer)
|
||||||
|
{
|
||||||
|
SerializationType val;
|
||||||
|
readIntBinary(val, buffer);
|
||||||
|
checkType(val);
|
||||||
|
has_additional_keys = (val & HasAdditionalKeysBit) != 0;
|
||||||
|
need_global_dictionary = (val & NeedGlobalDictionaryBit) != 0;
|
||||||
|
need_update_dictionary = (val & NeedUpdateDictionary) != 0;
|
||||||
|
type = static_cast<Type>(resetFlags(val));
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexesSerializationType(const IColumn & column,
|
||||||
|
bool has_additional_keys,
|
||||||
|
bool need_global_dictionary,
|
||||||
|
bool enumerate_dictionaries)
|
||||||
|
: has_additional_keys(has_additional_keys)
|
||||||
|
, need_global_dictionary(need_global_dictionary)
|
||||||
|
, need_update_dictionary(enumerate_dictionaries)
|
||||||
|
{
|
||||||
|
if (typeid_cast<const ColumnUInt8 *>(&column))
|
||||||
|
type = TUInt8;
|
||||||
|
else if (typeid_cast<const ColumnUInt16 *>(&column))
|
||||||
|
type = TUInt16;
|
||||||
|
else if (typeid_cast<const ColumnUInt32 *>(&column))
|
||||||
|
type = TUInt32;
|
||||||
|
else if (typeid_cast<const ColumnUInt64 *>(&column))
|
||||||
|
type = TUInt64;
|
||||||
|
else
|
||||||
|
throw Exception("Invalid Indexes column for IndexesSerializationType. Expected ColumnUInt*, got "
|
||||||
|
+ column.getName(), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
DataTypePtr getDataType() const
|
||||||
|
{
|
||||||
|
if (type == TUInt8)
|
||||||
|
return std::make_shared<DataTypeUInt8>();
|
||||||
|
if (type == TUInt16)
|
||||||
|
return std::make_shared<DataTypeUInt16>();
|
||||||
|
if (type == TUInt32)
|
||||||
|
return std::make_shared<DataTypeUInt32>();
|
||||||
|
if (type == TUInt64)
|
||||||
|
return std::make_shared<DataTypeUInt64>();
|
||||||
|
|
||||||
|
throw Exception("Can't create DataType from IndexesSerializationType.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexesSerializationType() = default;
|
||||||
|
};
|
||||||
|
|
||||||
|
struct SerializeStateWithDictionary : public IDataType::SerializeBinaryBulkState
|
||||||
|
{
|
||||||
|
KeysSerializationVersion key_version;
|
||||||
|
MutableColumnUniquePtr shared_dictionary;
|
||||||
|
|
||||||
|
explicit SerializeStateWithDictionary(UInt64 key_version) : key_version(key_version) {}
|
||||||
|
};
|
||||||
|
|
||||||
|
struct DeserializeStateWithDictionary : public IDataType::DeserializeBinaryBulkState
|
||||||
|
{
|
||||||
|
KeysSerializationVersion key_version;
|
||||||
|
ColumnUniquePtr global_dictionary;
|
||||||
|
|
||||||
|
IndexesSerializationType index_type;
|
||||||
|
ColumnPtr additional_keys;
|
||||||
|
UInt64 num_pending_rows = 0;
|
||||||
|
|
||||||
|
explicit DeserializeStateWithDictionary(UInt64 key_version) : key_version(key_version) {}
|
||||||
|
};
|
||||||
|
|
||||||
|
static SerializeStateWithDictionary * checkAndGetWithDictionarySerializeState(
|
||||||
|
IDataType::SerializeBinaryBulkStatePtr & state)
|
||||||
|
{
|
||||||
|
if (!state)
|
||||||
|
throw Exception("Got empty state for DataTypeWithDictionary.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto * with_dictionary_state = typeid_cast<SerializeStateWithDictionary *>(state.get());
|
||||||
|
if (!with_dictionary_state)
|
||||||
|
{
|
||||||
|
auto & state_ref = *state;
|
||||||
|
throw Exception("Invalid SerializeBinaryBulkState for DataTypeWithDictionary. Expected: "
|
||||||
|
+ demangle(typeid(SerializeStateWithDictionary).name()) + ", got "
|
||||||
|
+ demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
return with_dictionary_state;
|
||||||
|
}
|
||||||
|
|
||||||
|
static DeserializeStateWithDictionary * checkAndGetWithDictionaryDeserializeState(
|
||||||
|
IDataType::DeserializeBinaryBulkStatePtr & state)
|
||||||
|
{
|
||||||
|
if (!state)
|
||||||
|
throw Exception("Got empty state for DataTypeWithDictionary.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto * with_dictionary_state = typeid_cast<DeserializeStateWithDictionary *>(state.get());
|
||||||
|
if (!with_dictionary_state)
|
||||||
|
{
|
||||||
|
auto & state_ref = *state;
|
||||||
|
throw Exception("Invalid DeserializeBinaryBulkState for DataTypeWithDictionary. Expected: "
|
||||||
|
+ demangle(typeid(DeserializeStateWithDictionary).name()) + ", got "
|
||||||
|
+ demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
return with_dictionary_state;
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeWithDictionary::serializeBinaryBulkStatePrefix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
settings.path.push_back(Substream::DictionaryKeys);
|
||||||
|
auto * stream = settings.getter(settings.path);
|
||||||
|
settings.path.pop_back();
|
||||||
|
|
||||||
|
if (!stream)
|
||||||
|
throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStatePrefix",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
/// Write version and create SerializeBinaryBulkState.
|
||||||
|
UInt64 key_version = KeysSerializationVersion::SharedDictionariesWithAdditionalKeys;
|
||||||
|
|
||||||
|
writeIntBinary(key_version, *stream);
|
||||||
|
|
||||||
|
state = std::make_shared<SerializeStateWithDictionary>(key_version);
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeWithDictionary::serializeBinaryBulkStateSuffix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
auto * state_with_dictionary = checkAndGetWithDictionarySerializeState(state);
|
||||||
|
KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value);
|
||||||
|
|
||||||
|
if (state_with_dictionary->shared_dictionary && settings.low_cardinality_max_dictionary_size)
|
||||||
|
{
|
||||||
|
auto nested_column = state_with_dictionary->shared_dictionary->getNestedNotNullableColumn();
|
||||||
|
|
||||||
|
settings.path.push_back(Substream::DictionaryKeys);
|
||||||
|
auto * stream = settings.getter(settings.path);
|
||||||
|
settings.path.pop_back();
|
||||||
|
|
||||||
|
if (!stream)
|
||||||
|
throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStateSuffix",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
UInt64 num_keys = nested_column->size();
|
||||||
|
writeIntBinary(num_keys, *stream);
|
||||||
|
removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *stream, 0, num_keys);
|
||||||
|
state_with_dictionary->shared_dictionary = nullptr;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeWithDictionary::deserializeBinaryBulkStatePrefix(
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
settings.path.push_back(Substream::DictionaryKeys);
|
||||||
|
auto * stream = settings.getter(settings.path);
|
||||||
|
settings.path.pop_back();
|
||||||
|
|
||||||
|
if (!stream)
|
||||||
|
throw Exception("Got empty stream in DataTypeWithDictionary::deserializeBinaryBulkStatePrefix",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
UInt64 keys_version;
|
||||||
|
readIntBinary(keys_version, *stream);
|
||||||
|
|
||||||
|
state = std::make_shared<DeserializeStateWithDictionary>(keys_version);
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
template <typename T>
|
||||||
|
PaddedPODArray<T> * getIndexesData(IColumn & indexes)
|
||||||
|
{
|
||||||
|
auto * column = typeid_cast<ColumnVector<T> *>(&indexes);
|
||||||
|
if (column)
|
||||||
|
return &column->getData();
|
||||||
|
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
struct IndexMapsWithAdditionalKeys
|
||||||
|
{
|
||||||
|
MutableColumnPtr dictionary_map;
|
||||||
|
MutableColumnPtr additional_keys_map;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
IndexMapsWithAdditionalKeys mapIndexWithAdditionalKeysRef(PaddedPODArray<T> & index, size_t dict_size)
|
||||||
|
{
|
||||||
|
PaddedPODArray<T> copy(index.cbegin(), index.cend());
|
||||||
|
|
||||||
|
HashMap<T, T> dict_map;
|
||||||
|
HashMap<T, T> add_keys_map;
|
||||||
|
|
||||||
|
for (auto val : index)
|
||||||
|
{
|
||||||
|
if (val < dict_size)
|
||||||
|
dict_map.insert({val, dict_map.size()});
|
||||||
|
else
|
||||||
|
add_keys_map.insert({val, add_keys_map.size()});
|
||||||
|
}
|
||||||
|
|
||||||
|
auto dictionary_map = ColumnVector<T>::create(dict_map.size());
|
||||||
|
auto additional_keys_map = ColumnVector<T>::create(add_keys_map.size());
|
||||||
|
auto & dict_data = dictionary_map->getData();
|
||||||
|
auto & add_keys_data = additional_keys_map->getData();
|
||||||
|
|
||||||
|
for (auto val : dict_map)
|
||||||
|
dict_data[val.second] = val.first;
|
||||||
|
|
||||||
|
for (auto val : add_keys_map)
|
||||||
|
add_keys_data[val.second] = val.first - dict_size;
|
||||||
|
|
||||||
|
for (auto & val : index)
|
||||||
|
val = val < dict_size ? dict_map[val]
|
||||||
|
: add_keys_map[val] + dict_map.size();
|
||||||
|
|
||||||
|
for (size_t i = 0; i < index.size(); ++i)
|
||||||
|
{
|
||||||
|
T expected = index[i] < dict_data.size() ? dict_data[index[i]]
|
||||||
|
: add_keys_data[index[i] - dict_data.size()] + dict_size;
|
||||||
|
if (expected != copy[i])
|
||||||
|
throw Exception("Expected " + toString(expected) + ", but got " + toString(copy[i]), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
return {std::move(dictionary_map), std::move(additional_keys_map)};
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
IndexMapsWithAdditionalKeys mapIndexWithAdditionalKeys(PaddedPODArray<T> & index, size_t dict_size)
|
||||||
|
{
|
||||||
|
T max_less_dict_size = 0;
|
||||||
|
T max_value = 0;
|
||||||
|
|
||||||
|
auto size = index.size();
|
||||||
|
if (size == 0)
|
||||||
|
return {ColumnVector<T>::create(), ColumnVector<T>::create()};
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
auto val = index[i];
|
||||||
|
if (val < dict_size)
|
||||||
|
max_less_dict_size = std::max(max_less_dict_size, val);
|
||||||
|
|
||||||
|
max_value = std::max(max_value, val);
|
||||||
|
}
|
||||||
|
|
||||||
|
auto map_size = UInt64(max_less_dict_size) + 1;
|
||||||
|
auto overflow_map_size = max_value >= dict_size ? (UInt64(max_value - dict_size) + 1) : 0;
|
||||||
|
PaddedPODArray<T> map(map_size, 0);
|
||||||
|
PaddedPODArray<T> overflow_map(overflow_map_size, 0);
|
||||||
|
|
||||||
|
T zero_pos_value = 0;
|
||||||
|
T zero_pos_overflowed_value = 0;
|
||||||
|
UInt64 cur_pos = 0;
|
||||||
|
UInt64 cur_overflowed_pos = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
T val = index[i];
|
||||||
|
if (val < dict_size)
|
||||||
|
{
|
||||||
|
if (cur_pos == 0)
|
||||||
|
{
|
||||||
|
zero_pos_value = val;
|
||||||
|
++cur_pos;
|
||||||
|
}
|
||||||
|
else if (map[val] == 0 && val != zero_pos_value)
|
||||||
|
{
|
||||||
|
map[val] = cur_pos;
|
||||||
|
++cur_pos;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
T shifted_val = val - dict_size;
|
||||||
|
if (cur_overflowed_pos == 0)
|
||||||
|
{
|
||||||
|
zero_pos_overflowed_value = shifted_val;
|
||||||
|
++cur_overflowed_pos;
|
||||||
|
}
|
||||||
|
else if (overflow_map[shifted_val] == 0 && shifted_val != zero_pos_overflowed_value)
|
||||||
|
{
|
||||||
|
overflow_map[shifted_val] = cur_overflowed_pos;
|
||||||
|
++cur_overflowed_pos;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
auto dictionary_map = ColumnVector<T>::create(cur_pos);
|
||||||
|
auto additional_keys_map = ColumnVector<T>::create(cur_overflowed_pos);
|
||||||
|
auto & dict_data = dictionary_map->getData();
|
||||||
|
auto & add_keys_data = additional_keys_map->getData();
|
||||||
|
|
||||||
|
for (size_t i = 0; i < map_size; ++i)
|
||||||
|
if (map[i])
|
||||||
|
dict_data[map[i]] = static_cast<T>(i);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < overflow_map_size; ++i)
|
||||||
|
if (overflow_map[i])
|
||||||
|
add_keys_data[overflow_map[i]] = static_cast<T>(i);
|
||||||
|
|
||||||
|
if (cur_pos)
|
||||||
|
dict_data[0] = zero_pos_value;
|
||||||
|
if (cur_overflowed_pos)
|
||||||
|
add_keys_data[0] = zero_pos_overflowed_value;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
T & val = index[i];
|
||||||
|
if (val < dict_size)
|
||||||
|
val = map[val];
|
||||||
|
else
|
||||||
|
val = overflow_map[val - dict_size] + cur_pos;
|
||||||
|
}
|
||||||
|
|
||||||
|
return {std::move(dictionary_map), std::move(additional_keys_map)};
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Update column and return map with old indexes.
|
||||||
|
/// Let N is the number of distinct values which are less than max_size;
|
||||||
|
/// old_column - column before function call;
|
||||||
|
/// new_column - column after function call:
|
||||||
|
/// * if old_column[i] < max_size, than
|
||||||
|
/// dictionary_map[new_column[i]] = old_column[i]
|
||||||
|
/// * else
|
||||||
|
/// additional_keys_map[new_column[i]] = old_column[i] - dict_size + N
|
||||||
|
IndexMapsWithAdditionalKeys mapIndexWithAdditionalKeys(IColumn & column, size_t dict_size)
|
||||||
|
{
|
||||||
|
if (auto * data_uint8 = getIndexesData<UInt8>(column))
|
||||||
|
return mapIndexWithAdditionalKeys(*data_uint8, dict_size);
|
||||||
|
else if (auto * data_uint16 = getIndexesData<UInt16>(column))
|
||||||
|
return mapIndexWithAdditionalKeys(*data_uint16, dict_size);
|
||||||
|
else if (auto * data_uint32 = getIndexesData<UInt32>(column))
|
||||||
|
return mapIndexWithAdditionalKeys(*data_uint32, dict_size);
|
||||||
|
else if (auto * data_uint64 = getIndexesData<UInt64>(column))
|
||||||
|
return mapIndexWithAdditionalKeys(*data_uint64, dict_size);
|
||||||
|
else
|
||||||
|
throw Exception("Indexes column for mapIndexWithAdditionalKeys must be UInt, got" + column.getName(),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams(
|
||||||
|
const IColumn & column,
|
||||||
|
size_t offset,
|
||||||
|
size_t limit,
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
settings.path.push_back(Substream::DictionaryKeys);
|
||||||
|
auto * keys_stream = settings.getter(settings.path);
|
||||||
|
settings.path.back() = Substream::DictionaryIndexes;
|
||||||
|
auto * indexes_stream = settings.getter(settings.path);
|
||||||
|
settings.path.pop_back();
|
||||||
|
|
||||||
|
if (!keys_stream && !indexes_stream)
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (!keys_stream)
|
||||||
|
throw Exception("Got empty stream for DataTypeWithDictionary keys.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
if (!indexes_stream)
|
||||||
|
throw Exception("Got empty stream for DataTypeWithDictionary indexes.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
const ColumnWithDictionary & column_with_dictionary = typeid_cast<const ColumnWithDictionary &>(column);
|
||||||
|
|
||||||
|
auto * state_with_dictionary = checkAndGetWithDictionarySerializeState(state);
|
||||||
|
auto & global_dictionary = state_with_dictionary->shared_dictionary;
|
||||||
|
KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value);
|
||||||
|
|
||||||
|
bool need_update_dictionary = global_dictionary == nullptr;
|
||||||
|
if (need_update_dictionary)
|
||||||
|
global_dictionary = createColumnUnique(*dictionary_type);
|
||||||
|
|
||||||
|
size_t max_limit = column.size() - offset;
|
||||||
|
limit = limit ? std::min(limit, max_limit) : max_limit;
|
||||||
|
|
||||||
|
auto sub_column = column_with_dictionary.cutAndCompact(offset, limit);
|
||||||
|
ColumnPtr positions = sub_column->getIndexesPtr();
|
||||||
|
ColumnPtr keys = sub_column->getDictionary().getNestedColumn();
|
||||||
|
|
||||||
|
if (settings.low_cardinality_max_dictionary_size)
|
||||||
|
{
|
||||||
|
/// Insert used_keys into global dictionary and update sub_index.
|
||||||
|
auto indexes_with_overflow = global_dictionary->uniqueInsertRangeWithOverflow(*keys, 0, keys->size(),
|
||||||
|
settings.low_cardinality_max_dictionary_size);
|
||||||
|
size_t max_size = settings.low_cardinality_max_dictionary_size + indexes_with_overflow.overflowed_keys->size();
|
||||||
|
ColumnWithDictionary::Index(indexes_with_overflow.indexes->getPtr()).check(max_size);
|
||||||
|
|
||||||
|
if (global_dictionary->size() > settings.low_cardinality_max_dictionary_size)
|
||||||
|
throw Exception("Got dictionary with size " + toString(global_dictionary->size()) +
|
||||||
|
" but max dictionary size is " + toString(settings.low_cardinality_max_dictionary_size),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
positions = indexes_with_overflow.indexes->index(*positions, 0);
|
||||||
|
keys = std::move(indexes_with_overflow.overflowed_keys);
|
||||||
|
|
||||||
|
if (global_dictionary->size() < settings.low_cardinality_max_dictionary_size && !keys->empty())
|
||||||
|
throw Exception("Has additional keys, but dict size is " + toString(global_dictionary->size()) +
|
||||||
|
" which is less then max dictionary size (" + toString(settings.low_cardinality_max_dictionary_size) + ")",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (auto nullable_keys = typeid_cast<const ColumnNullable *>(keys.get()))
|
||||||
|
keys = nullable_keys->getNestedColumnPtr();
|
||||||
|
|
||||||
|
bool need_additional_keys = !keys->empty();
|
||||||
|
bool need_dictionary = settings.low_cardinality_max_dictionary_size != 0;
|
||||||
|
bool need_write_dictionary = !settings.low_cardinality_use_single_dictionary_for_part
|
||||||
|
&& global_dictionary->size() >= settings.low_cardinality_max_dictionary_size;
|
||||||
|
|
||||||
|
IndexesSerializationType index_version(*positions, need_additional_keys, need_dictionary, need_update_dictionary);
|
||||||
|
index_version.serialize(*indexes_stream);
|
||||||
|
|
||||||
|
if (need_write_dictionary)
|
||||||
|
{
|
||||||
|
const auto & nested_column = global_dictionary->getNestedNotNullableColumn();
|
||||||
|
UInt64 num_keys = nested_column->size();
|
||||||
|
writeIntBinary(num_keys, *keys_stream);
|
||||||
|
removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *keys_stream, 0, num_keys);
|
||||||
|
state_with_dictionary->shared_dictionary = nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (need_additional_keys)
|
||||||
|
{
|
||||||
|
UInt64 num_keys = keys->size();
|
||||||
|
writeIntBinary(num_keys, *indexes_stream);
|
||||||
|
removeNullable(dictionary_type)->serializeBinaryBulk(*keys, *indexes_stream, 0, num_keys);
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt64 num_rows = positions->size();
|
||||||
|
writeIntBinary(num_rows, *indexes_stream);
|
||||||
|
index_version.getDataType()->serializeBinaryBulk(*positions, *indexes_stream, 0, num_rows);
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams(
|
||||||
|
IColumn & column,
|
||||||
|
size_t limit,
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const
|
||||||
|
{
|
||||||
|
ColumnWithDictionary & column_with_dictionary = typeid_cast<ColumnWithDictionary &>(column);
|
||||||
|
|
||||||
|
auto * state_with_dictionary = checkAndGetWithDictionaryDeserializeState(state);
|
||||||
|
KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value);
|
||||||
|
|
||||||
|
settings.path.push_back(Substream::DictionaryKeys);
|
||||||
|
auto * keys_stream = settings.getter(settings.path);
|
||||||
|
settings.path.back() = Substream::DictionaryIndexes;
|
||||||
|
auto * indexes_stream = settings.getter(settings.path);
|
||||||
|
settings.path.pop_back();
|
||||||
|
|
||||||
|
if (!keys_stream && !indexes_stream)
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (!keys_stream)
|
||||||
|
throw Exception("Got empty stream for DataTypeWithDictionary keys.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
if (!indexes_stream)
|
||||||
|
throw Exception("Got empty stream for DataTypeWithDictionary indexes.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
auto readDictionary = [this, state_with_dictionary, keys_stream]()
|
||||||
|
{
|
||||||
|
UInt64 num_keys;
|
||||||
|
readIntBinary(num_keys, *keys_stream);
|
||||||
|
|
||||||
|
auto keys_type = removeNullable(dictionary_type);
|
||||||
|
auto global_dict_keys = keys_type->createColumn();
|
||||||
|
keys_type->deserializeBinaryBulk(*global_dict_keys, *keys_stream, num_keys, 0);
|
||||||
|
|
||||||
|
auto column_unique = createColumnUnique(*dictionary_type, std::move(global_dict_keys));
|
||||||
|
state_with_dictionary->global_dictionary = std::move(column_unique);
|
||||||
|
};
|
||||||
|
|
||||||
|
auto readAdditionalKeys = [this, state_with_dictionary, indexes_stream]()
|
||||||
|
{
|
||||||
|
UInt64 num_keys;
|
||||||
|
readIntBinary(num_keys, *indexes_stream);
|
||||||
|
auto keys_type = removeNullable(dictionary_type);
|
||||||
|
auto additional_keys = keys_type->createColumn();
|
||||||
|
keys_type->deserializeBinaryBulk(*additional_keys, *indexes_stream, num_keys, 0);
|
||||||
|
state_with_dictionary->additional_keys = std::move(additional_keys);
|
||||||
|
};
|
||||||
|
|
||||||
|
auto readIndexes = [this, state_with_dictionary, indexes_stream, &column_with_dictionary](UInt64 num_rows)
|
||||||
|
{
|
||||||
|
auto indexes_type = state_with_dictionary->index_type.getDataType();
|
||||||
|
MutableColumnPtr indexes_column = indexes_type->createColumn();
|
||||||
|
indexes_type->deserializeBinaryBulk(*indexes_column, *indexes_stream, num_rows, 0);
|
||||||
|
|
||||||
|
auto & global_dictionary = state_with_dictionary->global_dictionary;
|
||||||
|
const auto & additional_keys = state_with_dictionary->additional_keys;
|
||||||
|
|
||||||
|
bool has_additional_keys = state_with_dictionary->index_type.has_additional_keys;
|
||||||
|
bool column_is_empty = column_with_dictionary.empty();
|
||||||
|
|
||||||
|
if (!state_with_dictionary->index_type.need_global_dictionary)
|
||||||
|
{
|
||||||
|
column_with_dictionary.insertRangeFromDictionaryEncodedColumn(*additional_keys, *indexes_column);
|
||||||
|
}
|
||||||
|
else if (!has_additional_keys)
|
||||||
|
{
|
||||||
|
if (column_is_empty)
|
||||||
|
column_with_dictionary.setSharedDictionary(global_dictionary);
|
||||||
|
|
||||||
|
auto local_column = ColumnWithDictionary::create(global_dictionary, std::move(indexes_column));
|
||||||
|
column_with_dictionary.insertRangeFrom(*local_column, 0, num_rows);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto maps = mapIndexWithAdditionalKeys(*indexes_column, global_dictionary->size());
|
||||||
|
|
||||||
|
ColumnWithDictionary::Index(maps.additional_keys_map->getPtr()).check(additional_keys->size());
|
||||||
|
|
||||||
|
ColumnWithDictionary::Index(indexes_column->getPtr()).check(
|
||||||
|
maps.dictionary_map->size() + maps.additional_keys_map->size());
|
||||||
|
|
||||||
|
auto used_keys = (*std::move(global_dictionary->getNestedColumn()->index(*maps.dictionary_map, 0))).mutate();
|
||||||
|
|
||||||
|
if (!maps.additional_keys_map->empty())
|
||||||
|
{
|
||||||
|
auto used_add_keys = additional_keys->index(*maps.additional_keys_map, 0);
|
||||||
|
|
||||||
|
if (dictionary_type->isNullable())
|
||||||
|
{
|
||||||
|
ColumnPtr null_map = ColumnUInt8::create(used_add_keys->size(), 0);
|
||||||
|
used_add_keys = ColumnNullable::create(used_add_keys, null_map);
|
||||||
|
}
|
||||||
|
|
||||||
|
used_keys->insertRangeFrom(*used_add_keys, 0, used_add_keys->size());
|
||||||
|
}
|
||||||
|
|
||||||
|
column_with_dictionary.insertRangeFromDictionaryEncodedColumn(*used_keys, *indexes_column);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
while (limit)
|
||||||
|
{
|
||||||
|
if (state_with_dictionary->num_pending_rows == 0)
|
||||||
|
{
|
||||||
|
if (indexes_stream->eof())
|
||||||
|
break;
|
||||||
|
|
||||||
|
auto & index_type = state_with_dictionary->index_type;
|
||||||
|
auto & global_dictionary = state_with_dictionary->global_dictionary;
|
||||||
|
|
||||||
|
index_type.deserialize(*indexes_stream);
|
||||||
|
|
||||||
|
if (index_type.need_global_dictionary && (!global_dictionary || index_type.need_update_dictionary))
|
||||||
|
readDictionary();
|
||||||
|
|
||||||
|
if (state_with_dictionary->index_type.has_additional_keys)
|
||||||
|
readAdditionalKeys();
|
||||||
|
else
|
||||||
|
state_with_dictionary->additional_keys = nullptr;
|
||||||
|
|
||||||
|
readIntBinary(state_with_dictionary->num_pending_rows, *indexes_stream);
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t num_rows_to_read = std::min(limit, state_with_dictionary->num_pending_rows);
|
||||||
|
readIndexes(num_rows_to_read);
|
||||||
|
limit -= num_rows_to_read;
|
||||||
|
state_with_dictionary->num_pending_rows -= num_rows_to_read;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void DataTypeWithDictionary::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||||
|
{
|
||||||
|
dictionary_type->serializeBinary(field, ostr);
|
||||||
|
}
|
||||||
|
void DataTypeWithDictionary::deserializeBinary(Field & field, ReadBuffer & istr) const
|
||||||
|
{
|
||||||
|
dictionary_type->deserializeBinary(field, istr);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ... Args>
|
||||||
|
void DataTypeWithDictionary::serializeImpl(
|
||||||
|
const IColumn & column, size_t row_num, WriteBuffer & ostr,
|
||||||
|
DataTypeWithDictionary::SerealizeFunctionPtr<Args ...> func, Args & ... args) const
|
||||||
|
{
|
||||||
|
auto & column_with_dictionary = getColumnWithDictionary(column);
|
||||||
|
size_t unique_row_number = column_with_dictionary.getIndexes().getUInt(row_num);
|
||||||
|
(dictionary_type.get()->*func)(*column_with_dictionary.getDictionary().getNestedColumn(), unique_row_number, ostr, std::forward<Args>(args)...);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename ... Args>
|
||||||
|
void DataTypeWithDictionary::deserializeImpl(
|
||||||
|
IColumn & column, ReadBuffer & istr,
|
||||||
|
DataTypeWithDictionary::DeserealizeFunctionPtr<Args ...> func, Args & ... args) const
|
||||||
|
{
|
||||||
|
auto & column_with_dictionary = getColumnWithDictionary(column);
|
||||||
|
auto temp_column = column_with_dictionary.getDictionary().getNestedColumn()->cloneEmpty();
|
||||||
|
|
||||||
|
(dictionary_type.get()->*func)(*temp_column, istr, std::forward<Args>(args)...);
|
||||||
|
|
||||||
|
column_with_dictionary.insertFromFullColumn(*temp_column, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
template <typename Creator>
|
||||||
|
struct CreateColumnVector
|
||||||
|
{
|
||||||
|
MutableColumnUniquePtr & column;
|
||||||
|
const IDataType & keys_type;
|
||||||
|
const Creator & creator;
|
||||||
|
|
||||||
|
CreateColumnVector(MutableColumnUniquePtr & column, const IDataType & keys_type, const Creator & creator)
|
||||||
|
: column(column), keys_type(keys_type), creator(creator)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename T, size_t>
|
||||||
|
void operator()()
|
||||||
|
{
|
||||||
|
if (typeid_cast<const DataTypeNumber<T> *>(&keys_type))
|
||||||
|
column = creator((ColumnVector<T> *)(nullptr));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename Creator>
|
||||||
|
MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDataType & keys_type,
|
||||||
|
const Creator & creator)
|
||||||
|
{
|
||||||
|
auto * type = &keys_type;
|
||||||
|
if (auto * nullable_type = typeid_cast<const DataTypeNullable *>(&keys_type))
|
||||||
|
type = nullable_type->getNestedType().get();
|
||||||
|
|
||||||
|
if (type->isString())
|
||||||
|
return creator((ColumnString *)(nullptr));
|
||||||
|
if (type->isFixedString())
|
||||||
|
return creator((ColumnFixedString *)(nullptr));
|
||||||
|
if (typeid_cast<const DataTypeDate *>(type))
|
||||||
|
return creator((ColumnVector<UInt16> *)(nullptr));
|
||||||
|
if (typeid_cast<const DataTypeDateTime *>(type))
|
||||||
|
return creator((ColumnVector<UInt32> *)(nullptr));
|
||||||
|
if (type->isNumber())
|
||||||
|
{
|
||||||
|
MutableColumnUniquePtr column;
|
||||||
|
TypeListNumbers::forEach(CreateColumnVector(column, *type, creator));
|
||||||
|
|
||||||
|
if (!column)
|
||||||
|
throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
|
||||||
|
throw Exception("Unexpected dictionary type for DataTypeWithDictionary: " + type->getName(),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type)
|
||||||
|
{
|
||||||
|
auto creator = [&](auto x)
|
||||||
|
{
|
||||||
|
using ColumnType = typename std::remove_pointer<decltype(x)>::type;
|
||||||
|
return ColumnUnique<ColumnType>::create(keys_type);
|
||||||
|
};
|
||||||
|
return createColumnUniqueImpl(keys_type, creator);
|
||||||
|
}
|
||||||
|
|
||||||
|
MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys)
|
||||||
|
{
|
||||||
|
auto creator = [&](auto x)
|
||||||
|
{
|
||||||
|
using ColumnType = typename std::remove_pointer<decltype(x)>::type;
|
||||||
|
return ColumnUnique<ColumnType>::create(std::move(keys), keys_type.isNullable());
|
||||||
|
};
|
||||||
|
return createColumnUniqueImpl(keys_type, creator);
|
||||||
|
}
|
||||||
|
|
||||||
|
MutableColumnPtr DataTypeWithDictionary::createColumn() const
|
||||||
|
{
|
||||||
|
MutableColumnPtr indexes = DataTypeUInt8().createColumn();
|
||||||
|
MutableColumnPtr dictionary = createColumnUnique(*dictionary_type);
|
||||||
|
return ColumnWithDictionary::create(std::move(dictionary), std::move(indexes));
|
||||||
|
}
|
||||||
|
|
||||||
|
bool DataTypeWithDictionary::equals(const IDataType & rhs) const
|
||||||
|
{
|
||||||
|
if (typeid(rhs) != typeid(*this))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
auto & rhs_with_dictionary = static_cast<const DataTypeWithDictionary &>(rhs);
|
||||||
|
return dictionary_type->equals(*rhs_with_dictionary.dictionary_type);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
static DataTypePtr create(const ASTPtr & arguments)
|
||||||
|
{
|
||||||
|
if (!arguments || arguments->children.size() != 1)
|
||||||
|
throw Exception("LowCardinality data type family must have single argument - type of elements",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
return std::make_shared<DataTypeWithDictionary>(DataTypeFactory::instance().get(arguments->children[0]));
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerDataTypeWithDictionary(DataTypeFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerDataType("LowCardinality", create);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
170
dbms/src/DataTypes/DataTypeWithDictionary.h
Normal file
170
dbms/src/DataTypes/DataTypeWithDictionary.h
Normal file
@ -0,0 +1,170 @@
|
|||||||
|
#pragma once
|
||||||
|
#include <DataTypes/IDataType.h>
|
||||||
|
#include <Columns/IColumnUnique.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class DataTypeWithDictionary : public IDataType
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
DataTypePtr dictionary_type;
|
||||||
|
|
||||||
|
public:
|
||||||
|
DataTypeWithDictionary(DataTypePtr dictionary_type_);
|
||||||
|
|
||||||
|
const DataTypePtr & getDictionaryType() const { return dictionary_type; }
|
||||||
|
|
||||||
|
String getName() const override
|
||||||
|
{
|
||||||
|
return "LowCardinality(" + dictionary_type->getName() + ")";
|
||||||
|
}
|
||||||
|
const char * getFamilyName() const override { return "LowCardinality"; }
|
||||||
|
|
||||||
|
void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override;
|
||||||
|
|
||||||
|
void serializeBinaryBulkStatePrefix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void serializeBinaryBulkStateSuffix(
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void deserializeBinaryBulkStatePrefix(
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void serializeBinaryBulkWithMultipleStreams(
|
||||||
|
const IColumn & column,
|
||||||
|
size_t offset,
|
||||||
|
size_t limit,
|
||||||
|
SerializeBinaryBulkSettings & settings,
|
||||||
|
SerializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void deserializeBinaryBulkWithMultipleStreams(
|
||||||
|
IColumn & column,
|
||||||
|
size_t limit,
|
||||||
|
DeserializeBinaryBulkSettings & settings,
|
||||||
|
DeserializeBinaryBulkStatePtr & state) const override;
|
||||||
|
|
||||||
|
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
|
||||||
|
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
|
||||||
|
|
||||||
|
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override
|
||||||
|
{
|
||||||
|
serializeImpl(column, row_num, ostr, &IDataType::serializeBinary);
|
||||||
|
}
|
||||||
|
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override
|
||||||
|
{
|
||||||
|
deserializeImpl(column, istr, &IDataType::deserializeBinary);
|
||||||
|
}
|
||||||
|
|
||||||
|
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
serializeImpl(column, row_num, ostr, &IDataType::serializeTextEscaped, settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
deserializeImpl(column, istr, &IDataType::deserializeTextEscaped, settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
serializeImpl(column, row_num, ostr, &IDataType::serializeTextQuoted, settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
deserializeImpl(column, istr, &IDataType::deserializeTextQuoted, settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
serializeImpl(column, row_num, ostr, &IDataType::serializeTextCSV, settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
deserializeImpl(column, istr, &IDataType::deserializeTextCSV, settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
serializeImpl(column, row_num, ostr, &IDataType::serializeText, settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
serializeImpl(column, row_num, ostr, &IDataType::serializeTextJSON, settings);
|
||||||
|
}
|
||||||
|
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
deserializeImpl(column, istr, &IDataType::deserializeTextJSON, settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||||
|
{
|
||||||
|
serializeImpl(column, row_num, ostr, &IDataType::serializeTextXML, settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
MutableColumnPtr createColumn() const override;
|
||||||
|
|
||||||
|
Field getDefault() const override { return dictionary_type->getDefault(); }
|
||||||
|
|
||||||
|
bool equals(const IDataType & rhs) const override;
|
||||||
|
|
||||||
|
bool isParametric() const override { return true; }
|
||||||
|
bool haveSubtypes() const override { return true; }
|
||||||
|
bool cannotBeStoredInTables() const override { return dictionary_type->cannotBeStoredInTables(); }
|
||||||
|
bool shouldAlignRightInPrettyFormats() const override { return dictionary_type->shouldAlignRightInPrettyFormats(); }
|
||||||
|
bool textCanContainOnlyValidUTF8() const override { return dictionary_type->textCanContainOnlyValidUTF8(); }
|
||||||
|
bool isComparable() const override { return dictionary_type->isComparable(); }
|
||||||
|
bool canBeComparedWithCollation() const override { return dictionary_type->canBeComparedWithCollation(); }
|
||||||
|
bool canBeUsedAsVersion() const override { return dictionary_type->canBeUsedAsVersion(); }
|
||||||
|
bool isSummable() const override { return dictionary_type->isSummable(); };
|
||||||
|
bool canBeUsedInBitOperations() const override { return dictionary_type->canBeUsedInBitOperations(); };
|
||||||
|
bool canBeUsedInBooleanContext() const override { return dictionary_type->canBeUsedInBooleanContext(); };
|
||||||
|
bool isNumber() const override { return false; }
|
||||||
|
bool isInteger() const override { return false; }
|
||||||
|
bool isUnsignedInteger() const override { return false; }
|
||||||
|
bool isDateOrDateTime() const override { return false; }
|
||||||
|
bool isValueRepresentedByNumber() const override { return dictionary_type->isValueRepresentedByNumber(); }
|
||||||
|
bool isValueRepresentedByInteger() const override { return dictionary_type->isValueRepresentedByInteger(); }
|
||||||
|
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||||
|
bool isString() const override { return false; }
|
||||||
|
bool isFixedString() const override { return false; }
|
||||||
|
bool haveMaximumSizeOfValue() const override { return dictionary_type->haveMaximumSizeOfValue(); }
|
||||||
|
size_t getMaximumSizeOfValueInMemory() const override { return dictionary_type->getMaximumSizeOfValueInMemory(); }
|
||||||
|
size_t getSizeOfValueInMemory() const override { return dictionary_type->getSizeOfValueInMemory(); }
|
||||||
|
bool isCategorial() const override { return false; }
|
||||||
|
bool isEnum() const override { return false; }
|
||||||
|
bool isNullable() const override { return false; }
|
||||||
|
bool onlyNull() const override { return false; }
|
||||||
|
bool withDictionary() const override { return true; }
|
||||||
|
|
||||||
|
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type);
|
||||||
|
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys);
|
||||||
|
|
||||||
|
private:
|
||||||
|
|
||||||
|
template <typename ... Args>
|
||||||
|
using SerealizeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, WriteBuffer &, Args & ...) const;
|
||||||
|
|
||||||
|
template <typename ... Args>
|
||||||
|
void serializeImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr,
|
||||||
|
SerealizeFunctionPtr<Args ...> func, Args & ... args) const;
|
||||||
|
|
||||||
|
template <typename ... Args>
|
||||||
|
using DeserealizeFunctionPtr = void (IDataType::*)(IColumn &, ReadBuffer &, Args & ...) const;
|
||||||
|
|
||||||
|
template <typename ... Args>
|
||||||
|
void deserializeImpl(IColumn & column, ReadBuffer & istr,
|
||||||
|
DeserealizeFunctionPtr<Args ...> func, Args & ... args) const;
|
||||||
|
|
||||||
|
template <typename Creator>
|
||||||
|
static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator);
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -97,6 +97,8 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy
|
|||||||
/// and name is encoded as a whole.
|
/// and name is encoded as a whole.
|
||||||
stream_name += "%2E" + escapeForFileName(elem.tuple_element_name);
|
stream_name += "%2E" + escapeForFileName(elem.tuple_element_name);
|
||||||
}
|
}
|
||||||
|
else if (elem.type == Substream::DictionaryKeys)
|
||||||
|
stream_name += ".dict";
|
||||||
}
|
}
|
||||||
return stream_name;
|
return stream_name;
|
||||||
}
|
}
|
||||||
|
@ -79,6 +79,9 @@ public:
|
|||||||
NullMap,
|
NullMap,
|
||||||
|
|
||||||
TupleElement,
|
TupleElement,
|
||||||
|
|
||||||
|
DictionaryKeys,
|
||||||
|
DictionaryIndexes,
|
||||||
};
|
};
|
||||||
Type type;
|
Type type;
|
||||||
|
|
||||||
@ -91,14 +94,64 @@ public:
|
|||||||
using SubstreamPath = std::vector<Substream>;
|
using SubstreamPath = std::vector<Substream>;
|
||||||
|
|
||||||
using StreamCallback = std::function<void(const SubstreamPath &)>;
|
using StreamCallback = std::function<void(const SubstreamPath &)>;
|
||||||
virtual void enumerateStreams(StreamCallback callback, SubstreamPath path) const
|
virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
|
||||||
{
|
{
|
||||||
callback(path);
|
callback(path);
|
||||||
}
|
}
|
||||||
|
void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); }
|
||||||
|
void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); }
|
||||||
|
|
||||||
using OutputStreamGetter = std::function<WriteBuffer*(const SubstreamPath &)>;
|
using OutputStreamGetter = std::function<WriteBuffer*(const SubstreamPath &)>;
|
||||||
using InputStreamGetter = std::function<ReadBuffer*(const SubstreamPath &)>;
|
using InputStreamGetter = std::function<ReadBuffer*(const SubstreamPath &)>;
|
||||||
|
|
||||||
|
struct SerializeBinaryBulkState
|
||||||
|
{
|
||||||
|
virtual ~SerializeBinaryBulkState() = default;
|
||||||
|
};
|
||||||
|
struct DeserializeBinaryBulkState
|
||||||
|
{
|
||||||
|
virtual ~DeserializeBinaryBulkState() = default;
|
||||||
|
};
|
||||||
|
|
||||||
|
using SerializeBinaryBulkStatePtr = std::shared_ptr<SerializeBinaryBulkState>;
|
||||||
|
using DeserializeBinaryBulkStatePtr = std::shared_ptr<DeserializeBinaryBulkState>;
|
||||||
|
|
||||||
|
struct SerializeBinaryBulkSettings
|
||||||
|
{
|
||||||
|
OutputStreamGetter getter;
|
||||||
|
SubstreamPath path;
|
||||||
|
|
||||||
|
size_t low_cardinality_max_dictionary_size = 0;
|
||||||
|
bool low_cardinality_use_single_dictionary_for_part = true;
|
||||||
|
|
||||||
|
bool position_independent_encoding = true;
|
||||||
|
};
|
||||||
|
|
||||||
|
struct DeserializeBinaryBulkSettings
|
||||||
|
{
|
||||||
|
InputStreamGetter getter;
|
||||||
|
SubstreamPath path;
|
||||||
|
|
||||||
|
bool position_independent_encoding = true;
|
||||||
|
/// If not zero, may be used to avoid reallocations while reading column of String type.
|
||||||
|
double avg_value_size_hint = 0;
|
||||||
|
};
|
||||||
|
|
||||||
|
/// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark.
|
||||||
|
virtual void serializeBinaryBulkStatePrefix(
|
||||||
|
SerializeBinaryBulkSettings & /*settings*/,
|
||||||
|
SerializeBinaryBulkStatePtr & /*state*/) const {}
|
||||||
|
|
||||||
|
/// Call after serializeBinaryBulkWithMultipleStreams chain to finish serialization.
|
||||||
|
virtual void serializeBinaryBulkStateSuffix(
|
||||||
|
SerializeBinaryBulkSettings & /*settings*/,
|
||||||
|
SerializeBinaryBulkStatePtr & /*state*/) const {}
|
||||||
|
|
||||||
|
/// Call before before deserializeBinaryBulkWithMultipleStreams chain to get DeserializeBinaryBulkStatePtr.
|
||||||
|
virtual void deserializeBinaryBulkStatePrefix(
|
||||||
|
DeserializeBinaryBulkSettings & /*settings*/,
|
||||||
|
DeserializeBinaryBulkStatePtr & /*state*/) const {}
|
||||||
|
|
||||||
/** 'offset' and 'limit' are used to specify range.
|
/** 'offset' and 'limit' are used to specify range.
|
||||||
* limit = 0 - means no limit.
|
* limit = 0 - means no limit.
|
||||||
* offset must be not greater than size of column.
|
* offset must be not greater than size of column.
|
||||||
@ -107,29 +160,24 @@ public:
|
|||||||
*/
|
*/
|
||||||
virtual void serializeBinaryBulkWithMultipleStreams(
|
virtual void serializeBinaryBulkWithMultipleStreams(
|
||||||
const IColumn & column,
|
const IColumn & column,
|
||||||
OutputStreamGetter getter,
|
|
||||||
size_t offset,
|
size_t offset,
|
||||||
size_t limit,
|
size_t limit,
|
||||||
bool /*position_independent_encoding*/,
|
SerializeBinaryBulkSettings & settings,
|
||||||
SubstreamPath path) const
|
SerializeBinaryBulkStatePtr & /*state*/) const
|
||||||
{
|
{
|
||||||
if (WriteBuffer * stream = getter(path))
|
if (WriteBuffer * stream = settings.getter(settings.path))
|
||||||
serializeBinaryBulk(column, *stream, offset, limit);
|
serializeBinaryBulk(column, *stream, offset, limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Read no more than limit values and append them into column.
|
/// Read no more than limit values and append them into column.
|
||||||
* avg_value_size_hint - if not zero, may be used to avoid reallocations while reading column of String type.
|
|
||||||
*/
|
|
||||||
virtual void deserializeBinaryBulkWithMultipleStreams(
|
virtual void deserializeBinaryBulkWithMultipleStreams(
|
||||||
IColumn & column,
|
IColumn & column,
|
||||||
InputStreamGetter getter,
|
|
||||||
size_t limit,
|
size_t limit,
|
||||||
double avg_value_size_hint,
|
DeserializeBinaryBulkSettings & settings,
|
||||||
bool /*position_independent_encoding*/,
|
DeserializeBinaryBulkStatePtr & /*state*/) const
|
||||||
SubstreamPath path) const
|
|
||||||
{
|
{
|
||||||
if (ReadBuffer * stream = getter(path))
|
if (ReadBuffer * stream = settings.getter(settings.path))
|
||||||
deserializeBinaryBulk(column, *stream, limit, avg_value_size_hint);
|
deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Override these methods for data types that require just single stream (most of data types).
|
/** Override these methods for data types that require just single stream (most of data types).
|
||||||
@ -359,6 +407,8 @@ public:
|
|||||||
*/
|
*/
|
||||||
virtual bool canBeInsideNullable() const { return false; }
|
virtual bool canBeInsideNullable() const { return false; }
|
||||||
|
|
||||||
|
virtual bool withDictionary() const { return false; }
|
||||||
|
|
||||||
|
|
||||||
/// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
|
/// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
|
||||||
static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint);
|
static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint);
|
||||||
|
@ -22,6 +22,7 @@ try
|
|||||||
size_t size = strlen(s) + 1;
|
size_t size = strlen(s) + 1;
|
||||||
DataTypeString data_type;
|
DataTypeString data_type;
|
||||||
|
|
||||||
|
|
||||||
{
|
{
|
||||||
auto column = ColumnString::create();
|
auto column = ColumnString::create();
|
||||||
ColumnString::Chars_t & data = column->getChars();
|
ColumnString::Chars_t & data = column->getChars();
|
||||||
@ -37,8 +38,14 @@ try
|
|||||||
|
|
||||||
WriteBufferFromFile out_buf("test");
|
WriteBufferFromFile out_buf("test");
|
||||||
|
|
||||||
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
|
IDataType::SerializeBinaryBulkStatePtr state;
|
||||||
|
settings.getter = [&](const IDataType::SubstreamPath &){ return &out_buf; };
|
||||||
|
|
||||||
stopwatch.restart();
|
stopwatch.restart();
|
||||||
data_type.serializeBinaryBulkWithMultipleStreams(*column, [&](const IDataType::SubstreamPath &){ return &out_buf; }, 0, 0, true, {});
|
data_type.serializeBinaryBulkStatePrefix(settings, state);
|
||||||
|
data_type.serializeBinaryBulkWithMultipleStreams(*column, 0, 0, settings, state);
|
||||||
|
data_type.serializeBinaryBulkStateSuffix(settings, state);
|
||||||
stopwatch.stop();
|
stopwatch.stop();
|
||||||
|
|
||||||
std::cout << "Writing, elapsed: " << stopwatch.elapsedSeconds() << std::endl;
|
std::cout << "Writing, elapsed: " << stopwatch.elapsedSeconds() << std::endl;
|
||||||
@ -49,8 +56,13 @@ try
|
|||||||
|
|
||||||
ReadBufferFromFile in_buf("test");
|
ReadBufferFromFile in_buf("test");
|
||||||
|
|
||||||
|
IDataType::DeserializeBinaryBulkSettings settings;
|
||||||
|
IDataType::DeserializeBinaryBulkStatePtr state;
|
||||||
|
settings.getter = [&](const IDataType::SubstreamPath &){ return &in_buf; };
|
||||||
|
|
||||||
stopwatch.restart();
|
stopwatch.restart();
|
||||||
data_type.deserializeBinaryBulkWithMultipleStreams(*column, [&](const IDataType::SubstreamPath &){ return &in_buf; }, n, 0, true, {});
|
data_type.deserializeBinaryBulkStatePrefix(settings, state);
|
||||||
|
data_type.deserializeBinaryBulkWithMultipleStreams(*column, n, settings, state);
|
||||||
stopwatch.stop();
|
stopwatch.stop();
|
||||||
|
|
||||||
std::cout << "Reading, elapsed: " << stopwatch.elapsedSeconds() << std::endl;
|
std::cout << "Reading, elapsed: " << stopwatch.elapsedSeconds() << std::endl;
|
||||||
|
@ -27,7 +27,12 @@ int main(int, char **)
|
|||||||
WriteBufferFromOStream out_buf(ostr);
|
WriteBufferFromOStream out_buf(ostr);
|
||||||
|
|
||||||
stopwatch.restart();
|
stopwatch.restart();
|
||||||
data_type.serializeBinaryBulkWithMultipleStreams(*column, [&](const IDataType::SubstreamPath &){ return &out_buf; }, 0, 0, true, {});
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
|
settings.getter = [&](const IDataType::SubstreamPath &){ return &out_buf; };
|
||||||
|
IDataType::SerializeBinaryBulkStatePtr state;
|
||||||
|
data_type.serializeBinaryBulkStatePrefix(settings, state);
|
||||||
|
data_type.serializeBinaryBulkWithMultipleStreams(*column, 0, 0, settings, state);
|
||||||
|
data_type.serializeBinaryBulkStateSuffix(settings, state);
|
||||||
stopwatch.stop();
|
stopwatch.stop();
|
||||||
|
|
||||||
std::cout << "Elapsed: " << stopwatch.elapsedSeconds() << std::endl;
|
std::cout << "Elapsed: " << stopwatch.elapsedSeconds() << std::endl;
|
||||||
|
@ -749,7 +749,7 @@ static bool castTypeToEither(const IDataType * type, F && f)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
template <template <typename, typename> class Op, typename Name>
|
template <template <typename, typename> class Op, typename Name, bool CanBeExecutedOnDefaultArguments = true>
|
||||||
class FunctionBinaryArithmetic : public IFunction
|
class FunctionBinaryArithmetic : public IFunction
|
||||||
{
|
{
|
||||||
const Context & context;
|
const Context & context;
|
||||||
@ -993,6 +993,8 @@ public:
|
|||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
bool canBeExecutedOnDefaultArguments() const override { return CanBeExecutedOnDefaultArguments; }
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
@ -1141,9 +1143,9 @@ using FunctionPlus = FunctionBinaryArithmetic<PlusImpl, NamePlus>;
|
|||||||
using FunctionMinus = FunctionBinaryArithmetic<MinusImpl, NameMinus>;
|
using FunctionMinus = FunctionBinaryArithmetic<MinusImpl, NameMinus>;
|
||||||
using FunctionMultiply = FunctionBinaryArithmetic<MultiplyImpl, NameMultiply>;
|
using FunctionMultiply = FunctionBinaryArithmetic<MultiplyImpl, NameMultiply>;
|
||||||
using FunctionDivideFloating = FunctionBinaryArithmetic<DivideFloatingImpl, NameDivideFloating>;
|
using FunctionDivideFloating = FunctionBinaryArithmetic<DivideFloatingImpl, NameDivideFloating>;
|
||||||
using FunctionDivideIntegral = FunctionBinaryArithmetic<DivideIntegralImpl, NameDivideIntegral>;
|
using FunctionDivideIntegral = FunctionBinaryArithmetic<DivideIntegralImpl, NameDivideIntegral, false>;
|
||||||
using FunctionDivideIntegralOrZero = FunctionBinaryArithmetic<DivideIntegralOrZeroImpl, NameDivideIntegralOrZero>;
|
using FunctionDivideIntegralOrZero = FunctionBinaryArithmetic<DivideIntegralOrZeroImpl, NameDivideIntegralOrZero>;
|
||||||
using FunctionModulo = FunctionBinaryArithmetic<ModuloImpl, NameModulo>;
|
using FunctionModulo = FunctionBinaryArithmetic<ModuloImpl, NameModulo, false>;
|
||||||
using FunctionNegate = FunctionUnaryArithmetic<NegateImpl, NameNegate, true>;
|
using FunctionNegate = FunctionUnaryArithmetic<NegateImpl, NameNegate, true>;
|
||||||
using FunctionAbs = FunctionUnaryArithmetic<AbsImpl, NameAbs, false>;
|
using FunctionAbs = FunctionUnaryArithmetic<AbsImpl, NameAbs, false>;
|
||||||
using FunctionBitAnd = FunctionBinaryArithmetic<BitAndImpl, NameBitAnd>;
|
using FunctionBitAnd = FunctionBinaryArithmetic<BitAndImpl, NameBitAnd>;
|
||||||
@ -1157,8 +1159,8 @@ using FunctionBitRotateRight = FunctionBinaryArithmetic<BitRotateRightImpl, Name
|
|||||||
using FunctionBitTest = FunctionBinaryArithmetic<BitTestImpl, NameBitTest>;
|
using FunctionBitTest = FunctionBinaryArithmetic<BitTestImpl, NameBitTest>;
|
||||||
using FunctionLeast = FunctionBinaryArithmetic<LeastImpl, NameLeast>;
|
using FunctionLeast = FunctionBinaryArithmetic<LeastImpl, NameLeast>;
|
||||||
using FunctionGreatest = FunctionBinaryArithmetic<GreatestImpl, NameGreatest>;
|
using FunctionGreatest = FunctionBinaryArithmetic<GreatestImpl, NameGreatest>;
|
||||||
using FunctionGCD = FunctionBinaryArithmetic<GCDImpl, NameGCD>;
|
using FunctionGCD = FunctionBinaryArithmetic<GCDImpl, NameGCD, false>;
|
||||||
using FunctionLCM = FunctionBinaryArithmetic<LCMImpl, NameLCM>;
|
using FunctionLCM = FunctionBinaryArithmetic<LCMImpl, NameLCM, false>;
|
||||||
/// Assumed to be injective for the purpose of query optimization, but in fact it is not injective because of possible overflow.
|
/// Assumed to be injective for the purpose of query optimization, but in fact it is not injective because of possible overflow.
|
||||||
using FunctionIntExp2 = FunctionUnaryArithmetic<IntExp2Impl, NameIntExp2, true>;
|
using FunctionIntExp2 = FunctionUnaryArithmetic<IntExp2Impl, NameIntExp2, true>;
|
||||||
using FunctionIntExp10 = FunctionUnaryArithmetic<IntExp10Impl, NameIntExp10, true>;
|
using FunctionIntExp10 = FunctionUnaryArithmetic<IntExp10Impl, NameIntExp10, true>;
|
||||||
|
@ -1339,7 +1339,7 @@ void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arg
|
|||||||
|| executeNumber<Float32>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
|| executeNumber<Float32>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||||
|| executeNumber<Float64>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
|| executeNumber<Float64>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||||
|| executeString(*inner_col, offsets, res_data, res_offsets, nullable_col)))
|
|| executeString(*inner_col, offsets, res_data, res_offsets, nullable_col)))
|
||||||
executeHashed(offsets, original_data_columns, res_data, res_offsets);
|
executeHashed(offsets, original_data_columns, res_data, res_offsets, nullable_col);
|
||||||
|
|
||||||
block.getByPosition(result).column = std::move(res_ptr);
|
block.getByPosition(result).column = std::move(res_ptr);
|
||||||
}
|
}
|
||||||
@ -1449,13 +1449,21 @@ void FunctionArrayDistinct::executeHashed(
|
|||||||
const ColumnArray::Offsets & offsets,
|
const ColumnArray::Offsets & offsets,
|
||||||
const ColumnRawPtrs & columns,
|
const ColumnRawPtrs & columns,
|
||||||
IColumn & res_data_col,
|
IColumn & res_data_col,
|
||||||
ColumnArray::Offsets & res_offsets)
|
ColumnArray::Offsets & res_offsets,
|
||||||
|
const ColumnNullable * nullable_col)
|
||||||
{
|
{
|
||||||
size_t count = columns.size();
|
size_t count = columns.size();
|
||||||
|
|
||||||
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||||
|
|
||||||
|
const PaddedPODArray<UInt8> * src_null_map = nullptr;
|
||||||
|
|
||||||
|
if (nullable_col)
|
||||||
|
{
|
||||||
|
src_null_map = &static_cast<const ColumnUInt8 *>(&nullable_col->getNullMapColumn())->getData();
|
||||||
|
}
|
||||||
|
|
||||||
Set set;
|
Set set;
|
||||||
size_t prev_off = 0;
|
size_t prev_off = 0;
|
||||||
for (size_t i = 0; i < offsets.size(); ++i)
|
for (size_t i = 0; i < offsets.size(); ++i)
|
||||||
@ -1465,7 +1473,7 @@ void FunctionArrayDistinct::executeHashed(
|
|||||||
for (size_t j = prev_off; j < off; ++j)
|
for (size_t j = prev_off; j < off; ++j)
|
||||||
{
|
{
|
||||||
auto hash = hash128(j, count, columns);
|
auto hash = hash128(j, count, columns);
|
||||||
if (set.find(hash) == set.end())
|
if (set.find(hash) == set.end() && (!nullable_col || (*src_null_map)[j] == 0))
|
||||||
{
|
{
|
||||||
set.insert(hash);
|
set.insert(hash);
|
||||||
res_data_col.insertFrom(*columns[0], j);
|
res_data_col.insertFrom(*columns[0], j);
|
||||||
|
@ -1255,7 +1255,8 @@ private:
|
|||||||
const ColumnArray::Offsets & offsets,
|
const ColumnArray::Offsets & offsets,
|
||||||
const ColumnRawPtrs & columns,
|
const ColumnRawPtrs & columns,
|
||||||
IColumn & res_data_col,
|
IColumn & res_data_col,
|
||||||
ColumnArray::Offsets & res_offsets);
|
ColumnArray::Offsets & res_offsets,
|
||||||
|
const ColumnNullable * nullable_col);
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -1111,7 +1111,7 @@ public:
|
|||||||
return std::make_shared<DataTypeUUID>();
|
return std::make_shared<DataTypeUUID>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
|
@ -36,6 +36,8 @@
|
|||||||
#include <Functions/FunctionsMiscellaneous.h>
|
#include <Functions/FunctionsMiscellaneous.h>
|
||||||
#include <Functions/FunctionsDateTime.h>
|
#include <Functions/FunctionsDateTime.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
|
#include <DataTypes/DataTypeWithDictionary.h>
|
||||||
|
#include <Columns/ColumnWithDictionary.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -728,6 +730,7 @@ public:
|
|||||||
|
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
bool canBeExecutedOnDefaultArguments() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -1245,6 +1248,7 @@ protected:
|
|||||||
|
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
@ -1273,7 +1277,8 @@ public:
|
|||||||
|
|
||||||
PreparedFunctionPtr prepare(const Block & /*sample_block*/) const override
|
PreparedFunctionPtr prepare(const Block & /*sample_block*/) const override
|
||||||
{
|
{
|
||||||
return std::make_shared<PreparedFunctionCast>(prepare(getArgumentTypes()[0], getReturnType()), name);
|
return std::make_shared<PreparedFunctionCast>(
|
||||||
|
prepareUnpackDictionaries(getArgumentTypes()[0], getReturnType()), name);
|
||||||
}
|
}
|
||||||
|
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
@ -1378,7 +1383,7 @@ private:
|
|||||||
throw Exception{"CAST AS Array can only be performed between same-dimensional array types or from String", ErrorCodes::TYPE_MISMATCH};
|
throw Exception{"CAST AS Array can only be performed between same-dimensional array types or from String", ErrorCodes::TYPE_MISMATCH};
|
||||||
|
|
||||||
/// Prepare nested type conversion
|
/// Prepare nested type conversion
|
||||||
const auto nested_function = prepare(from_nested_type, to_nested_type);
|
const auto nested_function = prepareUnpackDictionaries(from_nested_type, to_nested_type);
|
||||||
|
|
||||||
return [nested_function, from_nested_type, to_nested_type](
|
return [nested_function, from_nested_type, to_nested_type](
|
||||||
Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||||
@ -1432,7 +1437,7 @@ private:
|
|||||||
|
|
||||||
/// Create conversion wrapper for each element in tuple
|
/// Create conversion wrapper for each element in tuple
|
||||||
for (const auto & idx_type : ext::enumerate(from_type->getElements()))
|
for (const auto & idx_type : ext::enumerate(from_type->getElements()))
|
||||||
element_wrappers.push_back(prepare(idx_type.second, to_element_types[idx_type.first]));
|
element_wrappers.push_back(prepareUnpackDictionaries(idx_type.second, to_element_types[idx_type.first]));
|
||||||
|
|
||||||
return [element_wrappers, from_element_types, to_element_types]
|
return [element_wrappers, from_element_types, to_element_types]
|
||||||
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||||
@ -1576,16 +1581,11 @@ private:
|
|||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
WrapperType prepare(const DataTypePtr & from_type, const DataTypePtr & to_type) const
|
WrapperType prepareUnpackDictionaries(const DataTypePtr & from_type, const DataTypePtr & to_type) const
|
||||||
{
|
{
|
||||||
/// Determine whether pre-processing and/or post-processing must take place during conversion.
|
|
||||||
|
|
||||||
bool source_is_nullable = from_type->isNullable();
|
|
||||||
bool result_is_nullable = to_type->isNullable();
|
|
||||||
|
|
||||||
if (from_type->onlyNull())
|
if (from_type->onlyNull())
|
||||||
{
|
{
|
||||||
if (!result_is_nullable)
|
if (!to_type->isNullable())
|
||||||
throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE};
|
throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||||
|
|
||||||
return [](Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
|
return [](Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
|
||||||
@ -1595,6 +1595,88 @@ private:
|
|||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
const auto * from_with_dict = typeid_cast<const DataTypeWithDictionary *>(from_type.get());
|
||||||
|
const auto * to_with_dict = typeid_cast<const DataTypeWithDictionary *>(to_type.get());
|
||||||
|
const auto & from_nested = from_with_dict ? from_with_dict->getDictionaryType() : from_type;
|
||||||
|
const auto & to_nested = to_with_dict ? to_with_dict->getDictionaryType() : to_type;
|
||||||
|
|
||||||
|
auto wrapper = prepareRemoveNullable(from_nested, to_nested);
|
||||||
|
if (!from_with_dict && !to_with_dict)
|
||||||
|
return wrapper;
|
||||||
|
|
||||||
|
return [wrapper, from_with_dict, to_with_dict]
|
||||||
|
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||||
|
{
|
||||||
|
auto & arg = block.getByPosition(arguments[0]);
|
||||||
|
auto & res = block.getByPosition(result);
|
||||||
|
|
||||||
|
ColumnPtr res_indexes;
|
||||||
|
/// For some types default can't be casted (for example, String to Int). In that case convert column to full.
|
||||||
|
bool src_converted_to_full_column = false;
|
||||||
|
|
||||||
|
{
|
||||||
|
/// Replace argument and result columns (and types) to dictionary key columns (and types).
|
||||||
|
/// Call nested wrapper in order to cast dictionary keys. Then restore block.
|
||||||
|
auto prev_arg_col = arg.column;
|
||||||
|
auto prev_arg_type = arg.type;
|
||||||
|
auto prev_res_type = res.type;
|
||||||
|
|
||||||
|
auto tmp_rows_count = input_rows_count;
|
||||||
|
|
||||||
|
if (to_with_dict)
|
||||||
|
res.type = to_with_dict->getDictionaryType();
|
||||||
|
|
||||||
|
if (from_with_dict)
|
||||||
|
{
|
||||||
|
auto * col_with_dict = typeid_cast<const ColumnWithDictionary *>(prev_arg_col.get());
|
||||||
|
arg.column = col_with_dict->getDictionary().getNestedColumn();
|
||||||
|
arg.type = from_with_dict->getDictionaryType();
|
||||||
|
|
||||||
|
/// TODO: Make map with defaults conversion.
|
||||||
|
src_converted_to_full_column = !removeNullable(arg.type)->equals(*removeNullable(res.type));
|
||||||
|
if (src_converted_to_full_column)
|
||||||
|
arg.column = arg.column->index(col_with_dict->getIndexes(), 0);
|
||||||
|
else
|
||||||
|
res_indexes = col_with_dict->getIndexesPtr();
|
||||||
|
|
||||||
|
tmp_rows_count = arg.column->size();
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Perform the requested conversion.
|
||||||
|
wrapper(block, arguments, result, tmp_rows_count);
|
||||||
|
|
||||||
|
arg.column = prev_arg_col;
|
||||||
|
arg.type = prev_arg_type;
|
||||||
|
res.type = prev_res_type;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (to_with_dict)
|
||||||
|
{
|
||||||
|
auto res_column = to_with_dict->createColumn();
|
||||||
|
auto * col_with_dict = typeid_cast<ColumnWithDictionary *>(res_column.get());
|
||||||
|
|
||||||
|
if (from_with_dict && !src_converted_to_full_column)
|
||||||
|
{
|
||||||
|
auto res_keys = std::move(res.column);
|
||||||
|
col_with_dict->insertRangeFromDictionaryEncodedColumn(*res_keys, *res_indexes);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
col_with_dict->insertRangeFromFullColumn(*res.column, 0, res.column->size());
|
||||||
|
|
||||||
|
res.column = std::move(res_column);
|
||||||
|
}
|
||||||
|
else if (!src_converted_to_full_column)
|
||||||
|
res.column = res.column->index(*res_indexes, 0);
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
WrapperType prepareRemoveNullable(const DataTypePtr & from_type, const DataTypePtr & to_type) const
|
||||||
|
{
|
||||||
|
/// Determine whether pre-processing and/or post-processing must take place during conversion.
|
||||||
|
|
||||||
|
bool source_is_nullable = from_type->isNullable();
|
||||||
|
bool result_is_nullable = to_type->isNullable();
|
||||||
|
|
||||||
auto wrapper = prepareImpl(removeNullable(from_type), removeNullable(to_type), result_is_nullable);
|
auto wrapper = prepareImpl(removeNullable(from_type), removeNullable(to_type), result_is_nullable);
|
||||||
|
|
||||||
if (result_is_nullable)
|
if (result_is_nullable)
|
||||||
@ -1740,6 +1822,7 @@ protected:
|
|||||||
}
|
}
|
||||||
|
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
|
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
template <typename DataType>
|
template <typename DataType>
|
||||||
|
@ -1206,7 +1206,7 @@ public:
|
|||||||
return std::make_shared<DataTypeDateTime>();
|
return std::make_shared<DataTypeDateTime>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -1235,7 +1235,7 @@ public:
|
|||||||
return std::make_shared<DataTypeDate>();
|
return std::make_shared<DataTypeDate>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -1264,7 +1264,7 @@ public:
|
|||||||
return std::make_shared<DataTypeDate>();
|
return std::make_shared<DataTypeDate>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
|
@ -220,7 +220,7 @@ public:
|
|||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
{
|
{
|
||||||
@ -316,7 +316,7 @@ public:
|
|||||||
return std::make_shared<DataTypeUInt8>();
|
return std::make_shared<DataTypeUInt8>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
{
|
{
|
||||||
@ -452,7 +452,7 @@ public:
|
|||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
{
|
{
|
||||||
@ -728,7 +728,7 @@ public:
|
|||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
{
|
{
|
||||||
|
@ -94,7 +94,7 @@ private:
|
|||||||
return std::make_shared<DataTypeUInt8>();
|
return std::make_shared<DataTypeUInt8>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -259,7 +259,7 @@ private:
|
|||||||
return std::make_shared<DataTypeString>();
|
return std::make_shared<DataTypeString>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -489,7 +489,7 @@ private:
|
|||||||
return std::make_shared<DataTypeString>();
|
return std::make_shared<DataTypeString>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -755,7 +755,7 @@ private:
|
|||||||
return std::make_shared<DataType>();
|
return std::make_shared<DataType>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -1030,7 +1030,7 @@ private:
|
|||||||
return std::make_shared<DataType>();
|
return std::make_shared<DataType>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -1263,7 +1263,7 @@ private:
|
|||||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -1423,7 +1423,7 @@ private:
|
|||||||
return std::make_shared<DataTypeUInt8>();
|
return std::make_shared<DataTypeUInt8>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
|
@ -23,7 +23,7 @@ public:
|
|||||||
|
|
||||||
bool isVariadic() const override { return true; }
|
bool isVariadic() const override { return true; }
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
size_t getNumberOfArguments() const override { return 0; }
|
size_t getNumberOfArguments() const override { return 0; }
|
||||||
|
|
||||||
|
@ -11,6 +11,7 @@
|
|||||||
#include <Columns/ColumnString.h>
|
#include <Columns/ColumnString.h>
|
||||||
#include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
#include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
|
#include <Columns/ColumnWithDictionary.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
#include <Common/UnicodeBar.h>
|
#include <Common/UnicodeBar.h>
|
||||||
#include <Common/UTF8Helpers.h>
|
#include <Common/UTF8Helpers.h>
|
||||||
@ -24,6 +25,7 @@
|
|||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/DataTypeEnum.h>
|
#include <DataTypes/DataTypeEnum.h>
|
||||||
|
#include <DataTypes/DataTypeWithDictionary.h>
|
||||||
#include <DataTypes/NumberTraits.h>
|
#include <DataTypes/NumberTraits.h>
|
||||||
#include <Formats/FormatSettings.h>
|
#include <Formats/FormatSettings.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
@ -111,7 +113,7 @@ public:
|
|||||||
return std::make_shared<DataTypeString>();
|
return std::make_shared<DataTypeString>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -135,9 +137,9 @@ public:
|
|||||||
return name;
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -213,6 +215,7 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
|
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
|
||||||
|
|
||||||
size_t getNumberOfArguments() const override
|
size_t getNumberOfArguments() const override
|
||||||
{
|
{
|
||||||
@ -404,9 +407,9 @@ public:
|
|||||||
return name;
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -448,9 +451,9 @@ public:
|
|||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -497,9 +500,9 @@ public:
|
|||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -541,9 +544,9 @@ public:
|
|||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -914,9 +917,9 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
/** It could return many different values for single argument. */
|
/** It could return many different values for single argument. */
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -1317,7 +1320,7 @@ public:
|
|||||||
return std::make_shared<DataTypeUInt32>();
|
return std::make_shared<DataTypeUInt32>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -1354,7 +1357,7 @@ public:
|
|||||||
return std::make_shared<DataTypeString>();
|
return std::make_shared<DataTypeString>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
@ -1388,9 +1391,9 @@ public:
|
|||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -1557,7 +1560,7 @@ public:
|
|||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -1698,7 +1701,7 @@ public:
|
|||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
|
||||||
|
|
||||||
bool isDeterministic() override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||||
|
|
||||||
@ -1866,6 +1869,119 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
class FunctionToLowCardinality: public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "toLowCardinality";
|
||||||
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToLowCardinality>(); }
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override { return 1; }
|
||||||
|
|
||||||
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
|
{
|
||||||
|
if (arguments[0]->withDictionary())
|
||||||
|
return arguments[0];
|
||||||
|
|
||||||
|
return std::make_shared<DataTypeWithDictionary>(arguments[0]);
|
||||||
|
}
|
||||||
|
|
||||||
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
|
{
|
||||||
|
auto arg_num = arguments[0];
|
||||||
|
const auto & arg = block.getByPosition(arg_num);
|
||||||
|
auto & res = block.getByPosition(result);
|
||||||
|
|
||||||
|
if (arg.type->withDictionary())
|
||||||
|
res.column = arg.column;
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto column = res.type->createColumn();
|
||||||
|
typeid_cast<ColumnWithDictionary &>(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size());
|
||||||
|
res.column = std::move(column);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
class FunctionLowCardinalityIndexes: public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "lowCardinalityIndexes";
|
||||||
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionLowCardinalityIndexes>(); }
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override { return 1; }
|
||||||
|
|
||||||
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
|
{
|
||||||
|
auto * type = typeid_cast<const DataTypeWithDictionary *>(arguments[0].get());
|
||||||
|
if (!type)
|
||||||
|
throw Exception("First first argument of function lowCardinalityIndexes must be ColumnWithDictionary, but got"
|
||||||
|
+ arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return std::make_shared<DataTypeUInt64>();
|
||||||
|
}
|
||||||
|
|
||||||
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
|
{
|
||||||
|
auto arg_num = arguments[0];
|
||||||
|
const auto & arg = block.getByPosition(arg_num);
|
||||||
|
auto & res = block.getByPosition(result);
|
||||||
|
auto indexes_col = typeid_cast<const ColumnWithDictionary *>(arg.column.get())->getIndexesPtr();
|
||||||
|
auto new_indexes_col = ColumnUInt64::create(indexes_col->size());
|
||||||
|
auto & data = new_indexes_col->getData();
|
||||||
|
for (size_t i = 0; i < data.size(); ++i)
|
||||||
|
data[i] = indexes_col->getUInt(i);
|
||||||
|
|
||||||
|
res.column = std::move(new_indexes_col);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
class FunctionLowCardinalityKeys: public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "lowCardinalityKeys";
|
||||||
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionLowCardinalityKeys>(); }
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override { return 1; }
|
||||||
|
|
||||||
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
|
{
|
||||||
|
auto * type = typeid_cast<const DataTypeWithDictionary *>(arguments[0].get());
|
||||||
|
if (!type)
|
||||||
|
throw Exception("First first argument of function lowCardinalityKeys must be ColumnWithDictionary, but got"
|
||||||
|
+ arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return type->getDictionaryType();
|
||||||
|
}
|
||||||
|
|
||||||
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
|
{
|
||||||
|
auto arg_num = arguments[0];
|
||||||
|
const auto & arg = block.getByPosition(arg_num);
|
||||||
|
auto & res = block.getByPosition(result);
|
||||||
|
const auto * column_with_dictionary = typeid_cast<const ColumnWithDictionary *>(arg.column.get());
|
||||||
|
res.column = column_with_dictionary->getDictionary().getNestedColumn()->cloneResized(arg.column->size());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
std::string FunctionVersion::getVersion() const
|
std::string FunctionVersion::getVersion() const
|
||||||
{
|
{
|
||||||
return VERSION_STRING;
|
return VERSION_STRING;
|
||||||
@ -1915,5 +2031,9 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
|||||||
factory.registerFunction<FunctionRunningDifference>();
|
factory.registerFunction<FunctionRunningDifference>();
|
||||||
factory.registerFunction<FunctionRunningIncome>();
|
factory.registerFunction<FunctionRunningIncome>();
|
||||||
factory.registerFunction<FunctionFinalizeAggregation>();
|
factory.registerFunction<FunctionFinalizeAggregation>();
|
||||||
|
|
||||||
|
factory.registerFunction<FunctionToLowCardinality>();
|
||||||
|
factory.registerFunction<FunctionLowCardinalityIndexes>();
|
||||||
|
factory.registerFunction<FunctionLowCardinalityKeys>();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <IO/WriteBufferFromString.h>
|
#include <IO/WriteBufferFromString.h>
|
||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
#include <Columns/ColumnFunction.h>
|
#include <Columns/ColumnFunction.h>
|
||||||
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -138,7 +138,7 @@ public:
|
|||||||
|
|
||||||
bool isVariadic() const override { return true; }
|
bool isVariadic() const override { return true; }
|
||||||
size_t getNumberOfArguments() const override { return 0; }
|
size_t getNumberOfArguments() const override { return 0; }
|
||||||
bool isDeterministicInScopeOfQuery() override { return false; }
|
bool isDeterministicInScopeOfQuery() const override { return false; }
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
{
|
{
|
||||||
|
@ -5,6 +5,9 @@
|
|||||||
#include <DataTypes/DataTypeNothing.h>
|
#include <DataTypes/DataTypeNothing.h>
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
#include <DataTypes/Native.h>
|
#include <DataTypes/Native.h>
|
||||||
|
#include <DataTypes/DataTypeWithDictionary.h>
|
||||||
|
#include <DataTypes/getLeastSupertype.h>
|
||||||
|
#include <Columns/ColumnWithDictionary.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
#include <Functions/IFunction.h>
|
#include <Functions/IFunction.h>
|
||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
@ -187,7 +190,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo
|
|||||||
for (size_t i = 0; i < arguments_size; ++i)
|
for (size_t i = 0; i < arguments_size; ++i)
|
||||||
temporary_argument_numbers[i] = i;
|
temporary_argument_numbers[i] = i;
|
||||||
|
|
||||||
execute(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows());
|
executeWithoutColumnsWithDictionary(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows());
|
||||||
|
|
||||||
block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column, input_rows_count);
|
block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column, input_rows_count);
|
||||||
return true;
|
return true;
|
||||||
@ -211,7 +214,7 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co
|
|||||||
if (null_presence.has_nullable)
|
if (null_presence.has_nullable)
|
||||||
{
|
{
|
||||||
Block temporary_block = createBlockWithNestedColumns(block, args, result);
|
Block temporary_block = createBlockWithNestedColumns(block, args, result);
|
||||||
execute(temporary_block, args, result, temporary_block.rows());
|
executeWithoutColumnsWithDictionary(temporary_block, args, result, temporary_block.rows());
|
||||||
block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args,
|
block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args,
|
||||||
result, input_rows_count);
|
result, input_rows_count);
|
||||||
return true;
|
return true;
|
||||||
@ -220,7 +223,7 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
|
void PreparedFunctionImpl::executeWithoutColumnsWithDictionary(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
|
||||||
{
|
{
|
||||||
if (defaultImplementationForConstantArguments(block, args, result, input_rows_count))
|
if (defaultImplementationForConstantArguments(block, args, result, input_rows_count))
|
||||||
return;
|
return;
|
||||||
@ -231,6 +234,115 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si
|
|||||||
executeImpl(block, args, result, input_rows_count);
|
executeImpl(block, args, result, input_rows_count);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(Block & block, const ColumnNumbers & args,
|
||||||
|
bool can_be_executed_on_default_arguments)
|
||||||
|
{
|
||||||
|
size_t num_rows = 0;
|
||||||
|
ColumnPtr indexes;
|
||||||
|
|
||||||
|
for (auto arg : args)
|
||||||
|
{
|
||||||
|
ColumnWithTypeAndName & column = block.getByPosition(arg);
|
||||||
|
if (auto * column_with_dict = checkAndGetColumn<ColumnWithDictionary>(column.column.get()))
|
||||||
|
{
|
||||||
|
if (indexes)
|
||||||
|
throw Exception("Expected single dictionary argument for function.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
indexes = column_with_dict->getIndexesPtr();
|
||||||
|
num_rows = column_with_dict->getDictionary().size();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!indexes)
|
||||||
|
throw Exception("Expected column with dictionary for any function argument.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
for (auto arg : args)
|
||||||
|
{
|
||||||
|
ColumnWithTypeAndName & column = block.getByPosition(arg);
|
||||||
|
if (auto * column_const = checkAndGetColumn<ColumnConst>(column.column.get()))
|
||||||
|
column.column = column_const->cloneResized(num_rows);
|
||||||
|
else if (auto * column_with_dict = checkAndGetColumn<ColumnWithDictionary>(column.column.get()))
|
||||||
|
{
|
||||||
|
auto * type_with_dict = checkAndGetDataType<DataTypeWithDictionary>(column.type.get());
|
||||||
|
|
||||||
|
if (!type_with_dict)
|
||||||
|
throw Exception("Incompatible type for column with dictionary: " + column.type->getName(),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
if (can_be_executed_on_default_arguments)
|
||||||
|
column.column = column_with_dict->getDictionary().getNestedColumn();
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto dict_encoded = column_with_dict->getMinimalDictionaryEncodedColumn(0, column_with_dict->size());
|
||||||
|
column.column = dict_encoded.dictionary;
|
||||||
|
indexes = dict_encoded.indexes;
|
||||||
|
}
|
||||||
|
column.type = type_with_dict->getDictionaryType();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return indexes;
|
||||||
|
}
|
||||||
|
|
||||||
|
static void convertColumnsWithDictionaryToFull(Block & block, const ColumnNumbers & args)
|
||||||
|
{
|
||||||
|
for (auto arg : args)
|
||||||
|
{
|
||||||
|
ColumnWithTypeAndName & column = block.getByPosition(arg);
|
||||||
|
if (auto * column_with_dict = checkAndGetColumn<ColumnWithDictionary>(column.column.get()))
|
||||||
|
{
|
||||||
|
auto * type_with_dict = checkAndGetDataType<DataTypeWithDictionary>(column.type.get());
|
||||||
|
|
||||||
|
if (!type_with_dict)
|
||||||
|
throw Exception("Incompatible type for column with dictionary: " + column.type->getName(),
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
column.column = column_with_dict->convertToFullColumn();
|
||||||
|
column.type = type_with_dict->getDictionaryType();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
|
||||||
|
{
|
||||||
|
if (useDefaultImplementationForColumnsWithDictionary())
|
||||||
|
{
|
||||||
|
auto & res = block.safeGetByPosition(result);
|
||||||
|
Block block_without_dicts = block.cloneWithoutColumns();
|
||||||
|
|
||||||
|
for (auto arg : args)
|
||||||
|
block_without_dicts.safeGetByPosition(arg).column = block.safeGetByPosition(arg).column;
|
||||||
|
|
||||||
|
if (auto * res_type_with_dict = typeid_cast<const DataTypeWithDictionary *>(res.type.get()))
|
||||||
|
{
|
||||||
|
block_without_dicts.safeGetByPosition(result).type = res_type_with_dict->getDictionaryType();
|
||||||
|
ColumnPtr indexes = replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(
|
||||||
|
block_without_dicts, args, canBeExecutedOnDefaultArguments());
|
||||||
|
|
||||||
|
executeWithoutColumnsWithDictionary(block_without_dicts, args, result, block_without_dicts.rows());
|
||||||
|
|
||||||
|
auto res_column = res.type->createColumn();
|
||||||
|
auto * column_with_dictionary = typeid_cast<ColumnWithDictionary *>(res_column.get());
|
||||||
|
|
||||||
|
if (!column_with_dictionary)
|
||||||
|
throw Exception("Expected ColumnWithDictionary, got" + res_column->getName(), ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
const auto & keys = block_without_dicts.safeGetByPosition(result).column;
|
||||||
|
column_with_dictionary->insertRangeFromDictionaryEncodedColumn(*keys, *indexes);
|
||||||
|
|
||||||
|
res.column = std::move(res_column);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
convertColumnsWithDictionaryToFull(block_without_dicts, args);
|
||||||
|
executeWithoutColumnsWithDictionary(block_without_dicts, args, result, input_rows_count);
|
||||||
|
res.column = block_without_dicts.safeGetByPosition(result).column;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
executeWithoutColumnsWithDictionary(block, args, result, input_rows_count);
|
||||||
|
}
|
||||||
|
|
||||||
void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const
|
void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const
|
||||||
{
|
{
|
||||||
if (isVariadic())
|
if (isVariadic())
|
||||||
@ -244,19 +356,19 @@ void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) con
|
|||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & arguments) const
|
DataTypePtr FunctionBuilderImpl::getReturnTypeWithoutDictionary(const ColumnsWithTypeAndName & arguments) const
|
||||||
{
|
{
|
||||||
checkNumberOfArguments(arguments.size());
|
checkNumberOfArguments(arguments.size());
|
||||||
|
|
||||||
if (!arguments.empty() && useDefaultImplementationForNulls())
|
if (!arguments.empty() && useDefaultImplementationForNulls())
|
||||||
{
|
{
|
||||||
NullPresence null_presense = getNullPresense(arguments);
|
NullPresence null_presence = getNullPresense(arguments);
|
||||||
|
|
||||||
if (null_presense.has_null_constant)
|
if (null_presence.has_null_constant)
|
||||||
{
|
{
|
||||||
return makeNullable(std::make_shared<DataTypeNothing>());
|
return makeNullable(std::make_shared<DataTypeNothing>());
|
||||||
}
|
}
|
||||||
if (null_presense.has_nullable)
|
if (null_presence.has_nullable)
|
||||||
{
|
{
|
||||||
Block nested_block = createBlockWithNestedColumns(Block(arguments), ext::collection_cast<ColumnNumbers>(ext::range(0, arguments.size())));
|
Block nested_block = createBlockWithNestedColumns(Block(arguments), ext::collection_cast<ColumnNumbers>(ext::range(0, arguments.size())));
|
||||||
auto return_type = getReturnTypeImpl(ColumnsWithTypeAndName(nested_block.begin(), nested_block.end()));
|
auto return_type = getReturnTypeImpl(ColumnsWithTypeAndName(nested_block.begin(), nested_block.end()));
|
||||||
@ -334,4 +446,39 @@ llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const DataTypes
|
|||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & arguments) const
|
||||||
|
{
|
||||||
|
if (useDefaultImplementationForColumnsWithDictionary())
|
||||||
|
{
|
||||||
|
bool has_type_with_dictionary = false;
|
||||||
|
bool can_run_function_on_dictionary = true;
|
||||||
|
|
||||||
|
ColumnsWithTypeAndName args_without_dictionary(arguments);
|
||||||
|
|
||||||
|
for (ColumnWithTypeAndName & arg : args_without_dictionary)
|
||||||
|
{
|
||||||
|
if (arg.column && arg.column->isColumnConst())
|
||||||
|
continue;
|
||||||
|
|
||||||
|
if (auto * type_with_dictionary = typeid_cast<const DataTypeWithDictionary *>(arg.type.get()))
|
||||||
|
{
|
||||||
|
if (has_type_with_dictionary)
|
||||||
|
can_run_function_on_dictionary = false;
|
||||||
|
|
||||||
|
has_type_with_dictionary = true;
|
||||||
|
arg.type = type_with_dictionary->getDictionaryType();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
can_run_function_on_dictionary = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (canBeExecutedOnLowCardinalityDictionary() && has_type_with_dictionary && can_run_function_on_dictionary)
|
||||||
|
return std::make_shared<DataTypeWithDictionary>(getReturnTypeWithoutDictionary(args_without_dictionary));
|
||||||
|
else
|
||||||
|
return getReturnTypeWithoutDictionary(args_without_dictionary);
|
||||||
|
}
|
||||||
|
|
||||||
|
return getReturnTypeWithoutDictionary(arguments);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -68,15 +68,28 @@ protected:
|
|||||||
*/
|
*/
|
||||||
virtual bool useDefaultImplementationForConstants() const { return false; }
|
virtual bool useDefaultImplementationForConstants() const { return false; }
|
||||||
|
|
||||||
|
/** If function arguments has single column with dictionary and all other arguments are constants, call function on nested column.
|
||||||
|
* Otherwise, convert all columns with dictionary to ordinary columns.
|
||||||
|
* Returns ColumnWithDictionary if at least one argument is ColumnWithDictionary.
|
||||||
|
*/
|
||||||
|
virtual bool useDefaultImplementationForColumnsWithDictionary() const { return true; }
|
||||||
|
|
||||||
/** Some arguments could remain constant during this implementation.
|
/** Some arguments could remain constant during this implementation.
|
||||||
*/
|
*/
|
||||||
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
|
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
|
||||||
|
|
||||||
|
/** True if function can be called on default arguments (include Nullable's) and won't throw.
|
||||||
|
* Counterexample: modulo(0, 0)
|
||||||
|
*/
|
||||||
|
virtual bool canBeExecutedOnDefaultArguments() const { return true; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
bool defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result,
|
bool defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result,
|
||||||
size_t input_rows_count);
|
size_t input_rows_count);
|
||||||
bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result,
|
bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result,
|
||||||
size_t input_rows_count);
|
size_t input_rows_count);
|
||||||
|
void executeWithoutColumnsWithDictionary(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||||
|
size_t input_rows_count);
|
||||||
};
|
};
|
||||||
|
|
||||||
using ValuePlaceholders = std::vector<std::function<llvm::Value * ()>>;
|
using ValuePlaceholders = std::vector<std::function<llvm::Value * ()>>;
|
||||||
@ -159,9 +172,9 @@ public:
|
|||||||
* Example: now(). Another example: functions that work with periodically updated dictionaries.
|
* Example: now(). Another example: functions that work with periodically updated dictionaries.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
virtual bool isDeterministic() { return true; }
|
virtual bool isDeterministic() const { return true; }
|
||||||
|
|
||||||
virtual bool isDeterministicInScopeOfQuery() { return true; }
|
virtual bool isDeterministicInScopeOfQuery() const { return true; }
|
||||||
|
|
||||||
/** Lets you know if the function is monotonic in a range of values.
|
/** Lets you know if the function is monotonic in a range of values.
|
||||||
* This is used to work with the index in a sorted chunk of data.
|
* This is used to work with the index in a sorted chunk of data.
|
||||||
@ -266,12 +279,25 @@ protected:
|
|||||||
*/
|
*/
|
||||||
virtual bool useDefaultImplementationForNulls() const { return true; }
|
virtual bool useDefaultImplementationForNulls() const { return true; }
|
||||||
|
|
||||||
|
/** If useDefaultImplementationForNulls() is true, than change arguments for getReturnType() and buildImpl().
|
||||||
|
* If function arguments has types with dictionary, convert them to ordinary types.
|
||||||
|
* getReturnType returns ColumnWithDictionary if at least one argument type is ColumnWithDictionary.
|
||||||
|
*/
|
||||||
|
virtual bool useDefaultImplementationForColumnsWithDictionary() const { return true; }
|
||||||
|
|
||||||
|
/// If it isn't, will convert all ColumnWithDictionary arguments to full columns.
|
||||||
|
virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; }
|
||||||
|
|
||||||
virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const = 0;
|
virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const = 0;
|
||||||
|
|
||||||
virtual void getLambdaArgumentTypesImpl(DataTypes & /*arguments*/) const
|
virtual void getLambdaArgumentTypesImpl(DataTypes & /*arguments*/) const
|
||||||
{
|
{
|
||||||
throw Exception("Function " + getName() + " can't have lambda-expressions as arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Function " + getName() + " can't have lambda-expressions as arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeWithoutDictionary(const ColumnsWithTypeAndName & arguments) const;
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Previous function interface.
|
/// Previous function interface.
|
||||||
@ -286,7 +312,10 @@ public:
|
|||||||
/// Override this functions to change default implementation behavior. See details in IMyFunction.
|
/// Override this functions to change default implementation behavior. See details in IMyFunction.
|
||||||
bool useDefaultImplementationForNulls() const override { return true; }
|
bool useDefaultImplementationForNulls() const override { return true; }
|
||||||
bool useDefaultImplementationForConstants() const override { return false; }
|
bool useDefaultImplementationForConstants() const override { return false; }
|
||||||
|
bool useDefaultImplementationForColumnsWithDictionary() const override { return true; }
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; }
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; }
|
||||||
|
bool canBeExecutedOnDefaultArguments() const override { return true; }
|
||||||
|
bool canBeExecutedOnLowCardinalityDictionary() const override { return isDeterministicInScopeOfQuery(); }
|
||||||
|
|
||||||
using PreparedFunctionImpl::execute;
|
using PreparedFunctionImpl::execute;
|
||||||
using FunctionBuilderImpl::getReturnTypeImpl;
|
using FunctionBuilderImpl::getReturnTypeImpl;
|
||||||
@ -365,7 +394,9 @@ protected:
|
|||||||
}
|
}
|
||||||
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
|
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
|
||||||
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
|
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
|
||||||
|
bool useDefaultImplementationForColumnsWithDictionary() const final { return function->useDefaultImplementationForColumnsWithDictionary(); }
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); }
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); }
|
||||||
|
bool canBeExecutedOnDefaultArguments() const override { return function->canBeExecutedOnDefaultArguments(); }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
std::shared_ptr<IFunction> function;
|
std::shared_ptr<IFunction> function;
|
||||||
@ -396,9 +427,9 @@ public:
|
|||||||
|
|
||||||
bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); }
|
bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); }
|
||||||
|
|
||||||
bool isDeterministic() override { return function->isDeterministic(); }
|
bool isDeterministic() const override { return function->isDeterministic(); }
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override { return function->isDeterministicInScopeOfQuery(); }
|
bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); }
|
||||||
|
|
||||||
bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); }
|
bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); }
|
||||||
|
|
||||||
@ -431,6 +462,8 @@ protected:
|
|||||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); }
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); }
|
||||||
|
|
||||||
bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); }
|
bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); }
|
||||||
|
bool useDefaultImplementationForColumnsWithDictionary() const override { return function->useDefaultImplementationForColumnsWithDictionary(); }
|
||||||
|
bool canBeExecutedOnLowCardinalityDictionary() const override { return function->canBeExecutedOnLowCardinalityDictionary(); }
|
||||||
|
|
||||||
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
|
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
|
||||||
{
|
{
|
||||||
|
@ -618,6 +618,13 @@ void Context::checkDatabaseAccessRights(const std::string & database_name) const
|
|||||||
checkDatabaseAccessRightsImpl(database_name);
|
checkDatabaseAccessRightsImpl(database_name);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool Context::hasDatabaseAccessRights(const String & database_name) const
|
||||||
|
{
|
||||||
|
auto lock = getLock();
|
||||||
|
return client_info.current_user.empty() || (database_name == "system") ||
|
||||||
|
shared->security_manager->hasAccessToDatabase(client_info.current_user, database_name);
|
||||||
|
}
|
||||||
|
|
||||||
void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) const
|
void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) const
|
||||||
{
|
{
|
||||||
if (client_info.current_user.empty() || (database_name == "system"))
|
if (client_info.current_user.empty() || (database_name == "system"))
|
||||||
@ -1793,6 +1800,7 @@ std::shared_ptr<ActionLocksManager> Context::getActionLocksManager()
|
|||||||
return shared->action_locks_manager;
|
return shared->action_locks_manager;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
SessionCleaner::~SessionCleaner()
|
SessionCleaner::~SessionCleaner()
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
|
@ -180,6 +180,7 @@ public:
|
|||||||
bool isTableExist(const String & database_name, const String & table_name) const;
|
bool isTableExist(const String & database_name, const String & table_name) const;
|
||||||
bool isDatabaseExist(const String & database_name) const;
|
bool isDatabaseExist(const String & database_name) const;
|
||||||
bool isExternalTableExist(const String & table_name) const;
|
bool isExternalTableExist(const String & table_name) const;
|
||||||
|
bool hasDatabaseAccessRights(const String & database_name) const;
|
||||||
void assertTableExists(const String & database_name, const String & table_name) const;
|
void assertTableExists(const String & database_name, const String & table_name) const;
|
||||||
|
|
||||||
/** The parameter check_database_access_rights exists to not check the permissions of the database again,
|
/** The parameter check_database_access_rights exists to not check the permissions of the database again,
|
||||||
|
@ -476,7 +476,7 @@ public:
|
|||||||
|
|
||||||
PreparedFunctionPtr prepare(const Block &) const override { return std::make_shared<LLVMPreparedFunction>(name, context); }
|
PreparedFunctionPtr prepare(const Block &) const override { return std::make_shared<LLVMPreparedFunction>(name, context); }
|
||||||
|
|
||||||
bool isDeterministic() override
|
bool isDeterministic() const override
|
||||||
{
|
{
|
||||||
for (const auto & f : originals)
|
for (const auto & f : originals)
|
||||||
if (!f->isDeterministic())
|
if (!f->isDeterministic())
|
||||||
@ -484,7 +484,7 @@ public:
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() override
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
{
|
{
|
||||||
for (const auto & f : originals)
|
for (const auto & f : originals)
|
||||||
if (!f->isDeterministicInScopeOfQuery())
|
if (!f->isDeterministicInScopeOfQuery())
|
||||||
|
@ -53,6 +53,7 @@ namespace ErrorCodes
|
|||||||
extern const int UNKNOWN_DATABASE_ENGINE;
|
extern const int UNKNOWN_DATABASE_ENGINE;
|
||||||
extern const int DUPLICATE_COLUMN;
|
extern const int DUPLICATE_COLUMN;
|
||||||
extern const int READONLY;
|
extern const int READONLY;
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -348,6 +349,33 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void InterpreterCreateQuery::checkSupportedTypes(const ColumnsDescription & columns, const Context & context)
|
||||||
|
{
|
||||||
|
const auto & settings = context.getSettingsRef();
|
||||||
|
bool allow_low_cardinality = settings.allow_experimental_low_cardinality_type != 0;
|
||||||
|
|
||||||
|
if (allow_low_cardinality)
|
||||||
|
return;
|
||||||
|
|
||||||
|
auto check_types = [&](const NamesAndTypesList & list)
|
||||||
|
{
|
||||||
|
for (const auto & column : list)
|
||||||
|
{
|
||||||
|
if (!allow_low_cardinality && column.type && column.type->withDictionary())
|
||||||
|
{
|
||||||
|
String message = "Cannot create table with column " + column.name + " which type is "
|
||||||
|
+ column.type->getName() + " because LowCardinality type is not allowed. "
|
||||||
|
+ "Set setting allow_experimental_low_cardinality_type = 1 in order to allow it.";
|
||||||
|
throw Exception(message, ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
check_types(columns.ordinary);
|
||||||
|
check_types(columns.materialized);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
ColumnsDescription InterpreterCreateQuery::setColumns(
|
ColumnsDescription InterpreterCreateQuery::setColumns(
|
||||||
ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const
|
ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const
|
||||||
{
|
{
|
||||||
@ -487,6 +515,10 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
|||||||
/// Set and retrieve list of columns.
|
/// Set and retrieve list of columns.
|
||||||
ColumnsDescription columns = setColumns(create, as_select_sample, as_storage);
|
ColumnsDescription columns = setColumns(create, as_select_sample, as_storage);
|
||||||
|
|
||||||
|
/// Some column types may be not allowed according to settings.
|
||||||
|
if (!create.attach)
|
||||||
|
checkSupportedTypes(columns, context);
|
||||||
|
|
||||||
/// Set the table engine if it was not specified explicitly.
|
/// Set the table engine if it was not specified explicitly.
|
||||||
setEngine(create);
|
setEngine(create);
|
||||||
|
|
||||||
|
@ -46,6 +46,8 @@ public:
|
|||||||
|
|
||||||
/// Obtain information about columns, their types and default values, for case when columns in CREATE query is specified explicitly.
|
/// Obtain information about columns, their types and default values, for case when columns in CREATE query is specified explicitly.
|
||||||
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, const Context & context);
|
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, const Context & context);
|
||||||
|
/// Check that column types are allowed for usage in table according to settings.
|
||||||
|
static void checkSupportedTypes(const ColumnsDescription & columns, const Context & context);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
BlockIO createDatabase(ASTCreateQuery & create);
|
BlockIO createDatabase(ASTCreateQuery & create);
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
#include <DataStreams/CreatingSetsBlockInputStream.h>
|
#include <DataStreams/CreatingSetsBlockInputStream.h>
|
||||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||||
#include <DataStreams/ConcatBlockInputStream.h>
|
#include <DataStreams/ConcatBlockInputStream.h>
|
||||||
|
#include <DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h>
|
||||||
|
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
@ -775,7 +776,8 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre
|
|||||||
{
|
{
|
||||||
pipeline.transform([&](auto & stream)
|
pipeline.transform([&](auto & stream)
|
||||||
{
|
{
|
||||||
stream = std::make_shared<ExpressionBlockInputStream>(stream, expression);
|
stream = std::make_shared<ConvertColumnWithDictionaryToFullBlockInputStream>(
|
||||||
|
std::make_shared<ExpressionBlockInputStream>(stream, expression));
|
||||||
});
|
});
|
||||||
|
|
||||||
Names key_names;
|
Names key_names;
|
||||||
|
@ -268,6 +268,11 @@ struct Settings
|
|||||||
M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \
|
M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \
|
||||||
\
|
\
|
||||||
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \
|
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \
|
||||||
|
\
|
||||||
|
M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.") \
|
||||||
|
M(SettingBool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.") \
|
||||||
|
M(SettingBool, allow_experimental_low_cardinality_type, false, "Allows to create table with LowCardinality types.") \
|
||||||
|
\
|
||||||
M(SettingBool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query between local and remote ones according to load_balancing") \
|
M(SettingBool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query between local and remote ones according to load_balancing") \
|
||||||
M(SettingUInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.") \
|
M(SettingUInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.") \
|
||||||
|
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/NaNUtils.h>
|
#include <Common/NaNUtils.h>
|
||||||
#include <DataTypes/DataTypeUUID.h>
|
#include <DataTypes/DataTypeUUID.h>
|
||||||
|
#include <DataTypes/DataTypeWithDictionary.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -217,12 +218,10 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co
|
|||||||
if (from_type_hint && from_type_hint->equals(to_type))
|
if (from_type_hint && from_type_hint->equals(to_type))
|
||||||
return from_value;
|
return from_value;
|
||||||
|
|
||||||
if (to_type.isNullable())
|
if (auto * with_dict_type = typeid_cast<const DataTypeWithDictionary *>(&to_type))
|
||||||
{
|
return convertFieldToType(from_value, *with_dict_type->getDictionaryType(), from_type_hint);
|
||||||
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(to_type);
|
else if (auto * nullable_type = typeid_cast<const DataTypeNullable *>(&to_type))
|
||||||
const DataTypePtr & nested_type = nullable_type.getNestedType();
|
return convertFieldToTypeImpl(from_value, *nullable_type->getNestedType());
|
||||||
return convertFieldToTypeImpl(from_value, *nested_type);
|
|
||||||
}
|
|
||||||
else
|
else
|
||||||
return convertFieldToTypeImpl(from_value, to_type);
|
return convertFieldToTypeImpl(from_value, to_type);
|
||||||
}
|
}
|
||||||
|
@ -17,6 +17,7 @@ namespace DB
|
|||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int SYNTAX_ERROR;
|
extern const int SYNTAX_ERROR;
|
||||||
|
extern const int TOP_AND_LIMIT_TOGETHER;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -38,6 +39,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
ParserKeyword s_limit("LIMIT");
|
ParserKeyword s_limit("LIMIT");
|
||||||
ParserKeyword s_settings("SETTINGS");
|
ParserKeyword s_settings("SETTINGS");
|
||||||
ParserKeyword s_by("BY");
|
ParserKeyword s_by("BY");
|
||||||
|
ParserKeyword s_top("TOP");
|
||||||
|
ParserKeyword s_offset("OFFSET");
|
||||||
|
|
||||||
ParserNotEmptyExpressionList exp_list(false);
|
ParserNotEmptyExpressionList exp_list(false);
|
||||||
ParserNotEmptyExpressionList exp_list_for_with_clause(false, true); /// Set prefer_alias_to_column_name for each alias.
|
ParserNotEmptyExpressionList exp_list_for_with_clause(false, true); /// Set prefer_alias_to_column_name for each alias.
|
||||||
@ -62,6 +65,26 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
if (s_distinct.ignore(pos, expected))
|
if (s_distinct.ignore(pos, expected))
|
||||||
select_query->distinct = true;
|
select_query->distinct = true;
|
||||||
|
|
||||||
|
if (s_top.ignore(pos, expected))
|
||||||
|
{
|
||||||
|
ParserToken open_bracket(TokenType::OpeningRoundBracket);
|
||||||
|
ParserToken close_bracket(TokenType::ClosingRoundBracket);
|
||||||
|
ParserNumber num;
|
||||||
|
|
||||||
|
if (open_bracket.ignore(pos, expected))
|
||||||
|
{
|
||||||
|
if (!num.parse(pos, select_query->limit_length, expected))
|
||||||
|
return false;
|
||||||
|
if (!close_bracket.ignore(pos, expected))
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
if (!num.parse(pos, select_query->limit_length, expected))
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
if (!exp_list_for_select_clause.parse(pos, select_query->select_expression_list, expected))
|
if (!exp_list_for_select_clause.parse(pos, select_query->select_expression_list, expected))
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -120,6 +143,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
/// LIMIT length | LIMIT offset, length | LIMIT count BY expr-list
|
/// LIMIT length | LIMIT offset, length | LIMIT count BY expr-list
|
||||||
if (s_limit.ignore(pos, expected))
|
if (s_limit.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
|
if (select_query->limit_length)
|
||||||
|
throw Exception("Can not use TOP and LIMIT together", ErrorCodes::TOP_AND_LIMIT_TOGETHER);
|
||||||
|
|
||||||
ParserToken s_comma(TokenType::Comma);
|
ParserToken s_comma(TokenType::Comma);
|
||||||
ParserNumber num;
|
ParserNumber num;
|
||||||
|
|
||||||
@ -140,6 +166,11 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
if (!exp_list.parse(pos, select_query->limit_by_expression_list, expected))
|
if (!exp_list.parse(pos, select_query->limit_by_expression_list, expected))
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
else if (s_offset.ignore(pos, expected))
|
||||||
|
{
|
||||||
|
if (!num.parse(pos, select_query->limit_offset, expected))
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// LIMIT length | LIMIT offset, length
|
/// LIMIT length | LIMIT offset, length
|
||||||
|
@ -586,12 +586,13 @@ void MergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) con
|
|||||||
|
|
||||||
for (const NameAndTypePair & name_type : storage.getColumns().getAllPhysical())
|
for (const NameAndTypePair & name_type : storage.getColumns().getAllPhysical())
|
||||||
{
|
{
|
||||||
|
IDataType::SubstreamPath path;
|
||||||
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
||||||
{
|
{
|
||||||
Poco::File bin_file(getFullPath() + IDataType::getFileNameForStream(name_type.name, substream_path) + ".bin");
|
Poco::File bin_file(getFullPath() + IDataType::getFileNameForStream(name_type.name, substream_path) + ".bin");
|
||||||
if (bin_file.exists())
|
if (bin_file.exists())
|
||||||
column_to_size[name_type.name] += bin_file.getSize();
|
column_to_size[name_type.name] += bin_file.getSize();
|
||||||
}, {});
|
}, path);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -637,6 +638,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
|
|||||||
{
|
{
|
||||||
for (const NameAndTypePair & name_type : columns)
|
for (const NameAndTypePair & name_type : columns)
|
||||||
{
|
{
|
||||||
|
IDataType::SubstreamPath stream_path;
|
||||||
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
||||||
{
|
{
|
||||||
String file_name = IDataType::getFileNameForStream(name_type.name, substream_path);
|
String file_name = IDataType::getFileNameForStream(name_type.name, substream_path);
|
||||||
@ -648,7 +650,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
|
|||||||
if (!checksums.files.count(bin_file_name))
|
if (!checksums.files.count(bin_file_name))
|
||||||
throw Exception("No " + bin_file_name + " file checksum for column " + name + " in part " + path,
|
throw Exception("No " + bin_file_name + " file checksum for column " + name + " in part " + path,
|
||||||
ErrorCodes::NO_FILE_IN_DATA_PART);
|
ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||||
}, {});
|
}, stream_path);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -721,7 +723,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
|
|||||||
throw Exception("Part " + path + " is broken: marks have different sizes.",
|
throw Exception("Part " + path + " is broken: marks have different sizes.",
|
||||||
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
|
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
|
||||||
}
|
}
|
||||||
}, {});
|
});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -42,7 +42,7 @@ MergeTreeReader::MergeTreeReader(const String & path,
|
|||||||
clockid_t clock_type)
|
clockid_t clock_type)
|
||||||
: avg_value_size_hints(avg_value_size_hints), path(path), data_part(data_part), columns(columns)
|
: avg_value_size_hints(avg_value_size_hints), path(path), data_part(data_part), columns(columns)
|
||||||
, uncompressed_cache(uncompressed_cache), mark_cache(mark_cache), save_marks_in_cache(save_marks_in_cache), storage(storage)
|
, uncompressed_cache(uncompressed_cache), mark_cache(mark_cache), save_marks_in_cache(save_marks_in_cache), storage(storage)
|
||||||
, all_mark_ranges(all_mark_ranges), aio_threshold(aio_threshold), max_read_buffer_size(max_read_buffer_size)
|
, all_mark_ranges(all_mark_ranges), aio_threshold(aio_threshold), max_read_buffer_size(max_read_buffer_size), index_granularity(storage.index_granularity)
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
@ -200,6 +200,11 @@ MergeTreeReader::Stream::Stream(
|
|||||||
getMark(right).offset_in_compressed_file - getMark(all_mark_ranges[i].begin).offset_in_compressed_file);
|
getMark(right).offset_in_compressed_file - getMark(all_mark_ranges[i].begin).offset_in_compressed_file);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Avoid empty buffer. May happen while reading dictionary for DataTypeWithDictionary.
|
||||||
|
/// For example: part has single dictionary and all marks point to the same position.
|
||||||
|
if (max_mark_range == 0)
|
||||||
|
max_mark_range = max_read_buffer_size;
|
||||||
|
|
||||||
size_t buffer_size = std::min(max_read_buffer_size, max_mark_range);
|
size_t buffer_size = std::min(max_read_buffer_size, max_mark_range);
|
||||||
|
|
||||||
/// Estimate size of the data to be read.
|
/// Estimate size of the data to be read.
|
||||||
@ -329,6 +334,26 @@ void MergeTreeReader::Stream::seekToMark(size_t index)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void MergeTreeReader::Stream::seekToStart()
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
if (cached_buffer)
|
||||||
|
cached_buffer->seek(0, 0);
|
||||||
|
if (non_cached_buffer)
|
||||||
|
non_cached_buffer->seek(0, 0);
|
||||||
|
}
|
||||||
|
catch (Exception & e)
|
||||||
|
{
|
||||||
|
/// Better diagnostics.
|
||||||
|
if (e.code() == ErrorCodes::ARGUMENT_OUT_OF_BOUND)
|
||||||
|
e.addMessage("(while seeking to start of column " + path_prefix + ")");
|
||||||
|
|
||||||
|
throw;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void MergeTreeReader::addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges,
|
void MergeTreeReader::addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges,
|
||||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type)
|
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type)
|
||||||
{
|
{
|
||||||
@ -353,7 +378,8 @@ void MergeTreeReader::addStreams(const String & name, const IDataType & type, co
|
|||||||
uncompressed_cache, aio_threshold, max_read_buffer_size, profile_callback, clock_type));
|
uncompressed_cache, aio_threshold, max_read_buffer_size, profile_callback, clock_type));
|
||||||
};
|
};
|
||||||
|
|
||||||
type.enumerateStreams(callback, {});
|
IDataType::SubstreamPath path;
|
||||||
|
type.enumerateStreams(callback, path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -362,28 +388,47 @@ void MergeTreeReader::readData(
|
|||||||
size_t from_mark, bool continue_reading, size_t max_rows_to_read,
|
size_t from_mark, bool continue_reading, size_t max_rows_to_read,
|
||||||
bool with_offsets)
|
bool with_offsets)
|
||||||
{
|
{
|
||||||
IDataType::InputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer *
|
auto get_stream_getter = [&](bool stream_for_prefix) -> IDataType::InputStreamGetter
|
||||||
{
|
{
|
||||||
/// If offsets for arrays have already been read.
|
return [&, stream_for_prefix](const IDataType::SubstreamPath & path) -> ReadBuffer *
|
||||||
if (!with_offsets && path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes)
|
{
|
||||||
return nullptr;
|
/// If offsets for arrays have already been read.
|
||||||
|
if (!with_offsets && path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes)
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
String stream_name = IDataType::getFileNameForStream(name, path);
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
||||||
|
|
||||||
auto it = streams.find(stream_name);
|
auto it = streams.find(stream_name);
|
||||||
if (it == streams.end())
|
if (it == streams.end())
|
||||||
return nullptr;
|
return nullptr;
|
||||||
|
|
||||||
Stream & stream = *it->second;
|
Stream & stream = *it->second;
|
||||||
|
|
||||||
if (!continue_reading)
|
if (stream_for_prefix)
|
||||||
stream.seekToMark(from_mark);
|
{
|
||||||
|
stream.seekToStart();
|
||||||
|
continue_reading = false;
|
||||||
|
}
|
||||||
|
else if (!continue_reading)
|
||||||
|
stream.seekToMark(from_mark);
|
||||||
|
|
||||||
return stream.data_buffer;
|
return stream.data_buffer;
|
||||||
|
};
|
||||||
};
|
};
|
||||||
|
|
||||||
double & avg_value_size_hint = avg_value_size_hints[name];
|
double & avg_value_size_hint = avg_value_size_hints[name];
|
||||||
type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, avg_value_size_hint, true, {});
|
IDataType::DeserializeBinaryBulkSettings settings;
|
||||||
|
settings.avg_value_size_hint = avg_value_size_hint;
|
||||||
|
|
||||||
|
if (deserialize_binary_bulk_state_map.count(name) == 0)
|
||||||
|
{
|
||||||
|
settings.getter = get_stream_getter(true);
|
||||||
|
type.deserializeBinaryBulkStatePrefix(settings, deserialize_binary_bulk_state_map[name]);
|
||||||
|
}
|
||||||
|
|
||||||
|
settings.getter = get_stream_getter(false);
|
||||||
|
auto & deserialize_state = deserialize_binary_bulk_state_map[name];
|
||||||
|
type.deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_state);
|
||||||
IDataType::updateAvgValueSizeHint(column, avg_value_size_hint);
|
IDataType::updateAvgValueSizeHint(column, avg_value_size_hint);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -23,6 +23,7 @@ class MergeTreeReader : private boost::noncopyable
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
using ValueSizeMap = std::map<std::string, double>;
|
using ValueSizeMap = std::map<std::string, double>;
|
||||||
|
using DeserializeBinaryBulkStateMap = std::map<std::string, IDataType::DeserializeBinaryBulkStatePtr>;
|
||||||
|
|
||||||
MergeTreeReader(const String & path, /// Path to the directory containing the part
|
MergeTreeReader(const String & path, /// Path to the directory containing the part
|
||||||
const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns,
|
const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns,
|
||||||
@ -63,6 +64,7 @@ private:
|
|||||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type);
|
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type);
|
||||||
|
|
||||||
void seekToMark(size_t index);
|
void seekToMark(size_t index);
|
||||||
|
void seekToStart();
|
||||||
|
|
||||||
ReadBuffer * data_buffer;
|
ReadBuffer * data_buffer;
|
||||||
|
|
||||||
@ -91,6 +93,8 @@ private:
|
|||||||
|
|
||||||
/// avg_value_size_hints are used to reduce the number of reallocations when creating columns of variable size.
|
/// avg_value_size_hints are used to reduce the number of reallocations when creating columns of variable size.
|
||||||
ValueSizeMap avg_value_size_hints;
|
ValueSizeMap avg_value_size_hints;
|
||||||
|
/// Stores states for IDataType::deserializeBinaryBulk
|
||||||
|
DeserializeBinaryBulkStateMap deserialize_binary_bulk_state_map;
|
||||||
String path;
|
String path;
|
||||||
MergeTreeData::DataPartPtr data_part;
|
MergeTreeData::DataPartPtr data_part;
|
||||||
|
|
||||||
@ -108,6 +112,7 @@ private:
|
|||||||
MarkRanges all_mark_ranges;
|
MarkRanges all_mark_ranges;
|
||||||
size_t aio_threshold;
|
size_t aio_threshold;
|
||||||
size_t max_read_buffer_size;
|
size_t max_read_buffer_size;
|
||||||
|
size_t index_granularity;
|
||||||
|
|
||||||
void addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges,
|
void addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges,
|
||||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type);
|
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type);
|
||||||
|
@ -64,7 +64,28 @@ void IMergedBlockOutputStream::addStreams(
|
|||||||
aio_threshold);
|
aio_threshold);
|
||||||
};
|
};
|
||||||
|
|
||||||
type.enumerateStreams(callback, {});
|
IDataType::SubstreamPath stream_path;
|
||||||
|
type.enumerateStreams(callback, stream_path);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
IDataType::OutputStreamGetter IMergedBlockOutputStream::createStreamGetter(
|
||||||
|
const String & name, OffsetColumns & offset_columns, bool skip_offsets)
|
||||||
|
{
|
||||||
|
return [&, skip_offsets] (const IDataType::SubstreamPath & substream_path) -> WriteBuffer *
|
||||||
|
{
|
||||||
|
bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes;
|
||||||
|
if (is_offsets && skip_offsets)
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
|
String stream_name = IDataType::getFileNameForStream(name, substream_path);
|
||||||
|
|
||||||
|
/// Don't write offsets more than one time for Nested type.
|
||||||
|
if (is_offsets && offset_columns.count(stream_name))
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
|
return &column_streams[stream_name]->compressed;
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -73,8 +94,15 @@ void IMergedBlockOutputStream::writeData(
|
|||||||
const IDataType & type,
|
const IDataType & type,
|
||||||
const IColumn & column,
|
const IColumn & column,
|
||||||
OffsetColumns & offset_columns,
|
OffsetColumns & offset_columns,
|
||||||
bool skip_offsets)
|
bool skip_offsets,
|
||||||
|
IDataType::SerializeBinaryBulkStatePtr & serialization_state)
|
||||||
{
|
{
|
||||||
|
auto & settings = storage.context.getSettingsRef();
|
||||||
|
IDataType::SerializeBinaryBulkSettings serialize_settings;
|
||||||
|
serialize_settings.getter = createStreamGetter(name, offset_columns, skip_offsets);
|
||||||
|
serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size;
|
||||||
|
serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part != 0;
|
||||||
|
|
||||||
size_t size = column.size();
|
size_t size = column.size();
|
||||||
size_t prev_mark = 0;
|
size_t prev_mark = 0;
|
||||||
while (prev_mark < size)
|
while (prev_mark < size)
|
||||||
@ -109,25 +137,10 @@ void IMergedBlockOutputStream::writeData(
|
|||||||
|
|
||||||
writeIntBinary(stream.plain_hashing.count(), stream.marks);
|
writeIntBinary(stream.plain_hashing.count(), stream.marks);
|
||||||
writeIntBinary(stream.compressed.offset(), stream.marks);
|
writeIntBinary(stream.compressed.offset(), stream.marks);
|
||||||
}, {});
|
}, serialize_settings.path);
|
||||||
}
|
}
|
||||||
|
|
||||||
IDataType::OutputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & substream_path) -> WriteBuffer *
|
type.serializeBinaryBulkWithMultipleStreams(column, prev_mark, limit, serialize_settings, serialization_state);
|
||||||
{
|
|
||||||
bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes;
|
|
||||||
if (is_offsets && skip_offsets)
|
|
||||||
return nullptr;
|
|
||||||
|
|
||||||
String stream_name = IDataType::getFileNameForStream(name, substream_path);
|
|
||||||
|
|
||||||
/// Don't write offsets more than one time for Nested type.
|
|
||||||
if (is_offsets && offset_columns.count(stream_name))
|
|
||||||
return nullptr;
|
|
||||||
|
|
||||||
return &column_streams[stream_name]->compressed;
|
|
||||||
};
|
|
||||||
|
|
||||||
type.serializeBinaryBulkWithMultipleStreams(column, stream_getter, prev_mark, limit, true, {});
|
|
||||||
|
|
||||||
/// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one.
|
/// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one.
|
||||||
type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path)
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path)
|
||||||
@ -143,7 +156,7 @@ void IMergedBlockOutputStream::writeData(
|
|||||||
return;
|
return;
|
||||||
|
|
||||||
column_streams[stream_name]->compressed.nextIfAtEnd();
|
column_streams[stream_name]->compressed.nextIfAtEnd();
|
||||||
}, {});
|
}, serialize_settings.path);
|
||||||
|
|
||||||
prev_mark += limit;
|
prev_mark += limit;
|
||||||
}
|
}
|
||||||
@ -157,7 +170,7 @@ void IMergedBlockOutputStream::writeData(
|
|||||||
String stream_name = IDataType::getFileNameForStream(name, substream_path);
|
String stream_name = IDataType::getFileNameForStream(name, substream_path);
|
||||||
offset_columns.insert(stream_name);
|
offset_columns.insert(stream_name);
|
||||||
}
|
}
|
||||||
}, {});
|
}, serialize_settings.path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -284,6 +297,19 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
|
|||||||
const NamesAndTypesList * total_column_list,
|
const NamesAndTypesList * total_column_list,
|
||||||
MergeTreeData::DataPart::Checksums * additional_column_checksums)
|
MergeTreeData::DataPart::Checksums * additional_column_checksums)
|
||||||
{
|
{
|
||||||
|
/// Finish columns serialization.
|
||||||
|
auto & settings = storage.context.getSettingsRef();
|
||||||
|
IDataType::SerializeBinaryBulkSettings serialize_settings;
|
||||||
|
serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size;
|
||||||
|
serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part != 0;
|
||||||
|
OffsetColumns offset_columns;
|
||||||
|
auto it = columns_list.begin();
|
||||||
|
for (size_t i = 0; i < columns_list.size(); ++i, ++it)
|
||||||
|
{
|
||||||
|
serialize_settings.getter = createStreamGetter(it->name, offset_columns, false);
|
||||||
|
it->type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]);
|
||||||
|
}
|
||||||
|
|
||||||
if (!total_column_list)
|
if (!total_column_list)
|
||||||
total_column_list = &columns_list;
|
total_column_list = &columns_list;
|
||||||
|
|
||||||
@ -395,28 +421,44 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
|||||||
index_columns[i] = primary_columns[i].column->cloneEmpty();
|
index_columns[i] = primary_columns[i].column->cloneEmpty();
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Now write the data.
|
if (serialization_states.empty())
|
||||||
for (const auto & it : columns_list)
|
|
||||||
{
|
{
|
||||||
const ColumnWithTypeAndName & column = block.getByName(it.name);
|
serialization_states.reserve(columns_list.size());
|
||||||
|
OffsetColumns tmp_offset_columns;
|
||||||
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
|
|
||||||
|
for (const auto & col : columns_list)
|
||||||
|
{
|
||||||
|
settings.getter = createStreamGetter(col.name, tmp_offset_columns, false);
|
||||||
|
serialization_states.emplace_back(nullptr);
|
||||||
|
col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Now write the data.
|
||||||
|
auto it = columns_list.begin();
|
||||||
|
for (size_t i = 0; i < columns_list.size(); ++i, ++it)
|
||||||
|
{
|
||||||
|
const ColumnWithTypeAndName & column = block.getByName(it->name);
|
||||||
|
|
||||||
if (permutation)
|
if (permutation)
|
||||||
{
|
{
|
||||||
auto primary_column_it = primary_columns_name_to_position.find(it.name);
|
auto primary_column_it = primary_columns_name_to_position.find(it->name);
|
||||||
if (primary_columns_name_to_position.end() != primary_column_it)
|
if (primary_columns_name_to_position.end() != primary_column_it)
|
||||||
{
|
{
|
||||||
writeData(column.name, *column.type, *primary_columns[primary_column_it->second].column, offset_columns, false);
|
auto & primary_column = *primary_columns[primary_column_it->second].column;
|
||||||
|
writeData(column.name, *column.type, primary_column, offset_columns, false, serialization_states[i]);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
/// We rearrange the columns that are not included in the primary key here; Then the result is released - to save RAM.
|
/// We rearrange the columns that are not included in the primary key here; Then the result is released - to save RAM.
|
||||||
ColumnPtr permutted_column = column.column->permute(*permutation, 0);
|
ColumnPtr permuted_column = column.column->permute(*permutation, 0);
|
||||||
writeData(column.name, *column.type, *permutted_column, offset_columns, false);
|
writeData(column.name, *column.type, *permuted_column, offset_columns, false, serialization_states[i]);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
writeData(column.name, *column.type, *column.column, offset_columns, false);
|
writeData(column.name, *column.type, *column.column, offset_columns, false, serialization_states[i]);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -470,11 +512,21 @@ void MergedColumnOnlyOutputStream::write(const Block & block)
|
|||||||
if (!initialized)
|
if (!initialized)
|
||||||
{
|
{
|
||||||
column_streams.clear();
|
column_streams.clear();
|
||||||
|
serialization_states.clear();
|
||||||
|
serialization_states.reserve(block.columns());
|
||||||
|
OffsetColumns tmp_offset_columns;
|
||||||
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
|
|
||||||
for (size_t i = 0; i < block.columns(); ++i)
|
for (size_t i = 0; i < block.columns(); ++i)
|
||||||
{
|
{
|
||||||
addStreams(part_path, block.safeGetByPosition(i).name,
|
const auto & col = block.safeGetByPosition(i);
|
||||||
*block.safeGetByPosition(i).type, 0, skip_offsets);
|
|
||||||
|
addStreams(part_path, col.name, *col.type, 0, skip_offsets);
|
||||||
|
serialization_states.emplace_back(nullptr);
|
||||||
|
settings.getter = createStreamGetter(col.name, tmp_offset_columns, false);
|
||||||
|
col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back());
|
||||||
}
|
}
|
||||||
|
|
||||||
initialized = true;
|
initialized = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -484,7 +536,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block)
|
|||||||
for (size_t i = 0; i < block.columns(); ++i)
|
for (size_t i = 0; i < block.columns(); ++i)
|
||||||
{
|
{
|
||||||
const ColumnWithTypeAndName & column = block.safeGetByPosition(i);
|
const ColumnWithTypeAndName & column = block.safeGetByPosition(i);
|
||||||
writeData(column.name, *column.type, *column.column, offset_columns, skip_offsets);
|
writeData(column.name, *column.type, *column.column, offset_columns, skip_offsets, serialization_states[i]);
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity;
|
size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity;
|
||||||
@ -498,6 +550,19 @@ void MergedColumnOnlyOutputStream::writeSuffix()
|
|||||||
|
|
||||||
MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums()
|
MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums()
|
||||||
{
|
{
|
||||||
|
/// Finish columns serialization.
|
||||||
|
auto & settings = storage.context.getSettingsRef();
|
||||||
|
IDataType::SerializeBinaryBulkSettings serialize_settings;
|
||||||
|
serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size;
|
||||||
|
serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part != 0;
|
||||||
|
OffsetColumns offset_columns;
|
||||||
|
for (size_t i = 0; i < header.columns(); ++i)
|
||||||
|
{
|
||||||
|
auto & column = header.safeGetByPosition(i);
|
||||||
|
serialize_settings.getter = createStreamGetter(column.name, offset_columns, skip_offsets);
|
||||||
|
column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]);
|
||||||
|
}
|
||||||
|
|
||||||
MergeTreeData::DataPart::Checksums checksums;
|
MergeTreeData::DataPart::Checksums checksums;
|
||||||
|
|
||||||
for (auto & column_stream : column_streams)
|
for (auto & column_stream : column_streams)
|
||||||
@ -510,6 +575,7 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG
|
|||||||
}
|
}
|
||||||
|
|
||||||
column_streams.clear();
|
column_streams.clear();
|
||||||
|
serialization_states.clear();
|
||||||
initialized = false;
|
initialized = false;
|
||||||
|
|
||||||
return checksums;
|
return checksums;
|
||||||
|
@ -25,6 +25,8 @@ public:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
using OffsetColumns = std::set<std::string>;
|
using OffsetColumns = std::set<std::string>;
|
||||||
|
using SerializationState = IDataType::SerializeBinaryBulkStatePtr;
|
||||||
|
using SerializationStates = std::vector<SerializationState>;
|
||||||
|
|
||||||
struct ColumnStream
|
struct ColumnStream
|
||||||
{
|
{
|
||||||
@ -64,8 +66,12 @@ protected:
|
|||||||
|
|
||||||
void addStreams(const String & path, const String & name, const IDataType & type, size_t estimated_size, bool skip_offsets);
|
void addStreams(const String & path, const String & name, const IDataType & type, size_t estimated_size, bool skip_offsets);
|
||||||
|
|
||||||
|
|
||||||
|
IDataType::OutputStreamGetter createStreamGetter(const String & name, OffsetColumns & offset_columns, bool skip_offsets);
|
||||||
|
|
||||||
/// Write data of one column.
|
/// Write data of one column.
|
||||||
void writeData(const String & name, const IDataType & type, const IColumn & column, OffsetColumns & offset_columns, bool skip_offsets);
|
void writeData(const String & name, const IDataType & type, const IColumn & column, OffsetColumns & offset_columns,
|
||||||
|
bool skip_offsets, IDataType::SerializeBinaryBulkStatePtr & serialization_state);
|
||||||
|
|
||||||
MergeTreeData & storage;
|
MergeTreeData & storage;
|
||||||
|
|
||||||
@ -132,6 +138,7 @@ private:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
NamesAndTypesList columns_list;
|
NamesAndTypesList columns_list;
|
||||||
|
SerializationStates serialization_states;
|
||||||
String part_path;
|
String part_path;
|
||||||
|
|
||||||
size_t rows_count = 0;
|
size_t rows_count = 0;
|
||||||
@ -158,6 +165,7 @@ public:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
Block header;
|
Block header;
|
||||||
|
SerializationStates serialization_states;
|
||||||
String part_path;
|
String part_path;
|
||||||
|
|
||||||
bool initialized = false;
|
bool initialized = false;
|
||||||
|
@ -249,6 +249,8 @@ MergeTreeData::DataPart::Checksums checkDataPart(
|
|||||||
|
|
||||||
while (true)
|
while (true)
|
||||||
{
|
{
|
||||||
|
IDataType::DeserializeBinaryBulkSettings settings;
|
||||||
|
|
||||||
/// Check that mark points to current position in file.
|
/// Check that mark points to current position in file.
|
||||||
bool marks_eof = false;
|
bool marks_eof = false;
|
||||||
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
||||||
@ -270,7 +272,7 @@ MergeTreeData::DataPart::Checksums checkDataPart(
|
|||||||
+ ", mrk file offset: " + toString(stream.mrk_hashing_buf.count()));
|
+ ", mrk file offset: " + toString(stream.mrk_hashing_buf.count()));
|
||||||
throw;
|
throw;
|
||||||
}
|
}
|
||||||
}, {});
|
}, settings.path);
|
||||||
|
|
||||||
++mark_num;
|
++mark_num;
|
||||||
|
|
||||||
@ -278,18 +280,18 @@ MergeTreeData::DataPart::Checksums checkDataPart(
|
|||||||
/// NOTE Shared array sizes of Nested columns are read more than once. That's Ok.
|
/// NOTE Shared array sizes of Nested columns are read more than once. That's Ok.
|
||||||
|
|
||||||
MutableColumnPtr tmp_column = name_type.type->createColumn();
|
MutableColumnPtr tmp_column = name_type.type->createColumn();
|
||||||
name_type.type->deserializeBinaryBulkWithMultipleStreams(
|
settings.getter = [&](const IDataType::SubstreamPath & substream_path)
|
||||||
*tmp_column,
|
{
|
||||||
[&](const IDataType::SubstreamPath & substream_path)
|
String file_name = IDataType::getFileNameForStream(name_type.name, substream_path);
|
||||||
{
|
auto stream_it = streams.find(file_name);
|
||||||
String file_name = IDataType::getFileNameForStream(name_type.name, substream_path);
|
if (stream_it == streams.end())
|
||||||
auto stream_it = streams.find(file_name);
|
throw Exception("Logical error: cannot find stream " + file_name);
|
||||||
if (stream_it == streams.end())
|
return &stream_it->second.uncompressed_hashing_buf;
|
||||||
throw Exception("Logical error: cannot find stream " + file_name);
|
};
|
||||||
return &stream_it->second.uncompressed_hashing_buf;
|
|
||||||
},
|
IDataType::DeserializeBinaryBulkStatePtr state;
|
||||||
index_granularity,
|
name_type.type->deserializeBinaryBulkStatePrefix(settings, state);
|
||||||
0, true, {});
|
name_type.type->deserializeBinaryBulkWithMultipleStreams(*tmp_column, index_granularity, settings, state);
|
||||||
|
|
||||||
size_t read_size = tmp_column->size();
|
size_t read_size = tmp_column->size();
|
||||||
column_size += read_size;
|
column_size += read_size;
|
||||||
|
@ -76,6 +76,7 @@ protected:
|
|||||||
Block readImpl() override;
|
Block readImpl() override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
|
||||||
size_t block_size;
|
size_t block_size;
|
||||||
NamesAndTypesList columns;
|
NamesAndTypesList columns;
|
||||||
StorageLog & storage;
|
StorageLog & storage;
|
||||||
@ -101,6 +102,10 @@ private:
|
|||||||
using FileStreams = std::map<std::string, Stream>;
|
using FileStreams = std::map<std::string, Stream>;
|
||||||
FileStreams streams;
|
FileStreams streams;
|
||||||
|
|
||||||
|
using DeserializeState = IDataType::DeserializeBinaryBulkStatePtr;
|
||||||
|
using DeserializeStates = std::map<String, DeserializeState>;
|
||||||
|
DeserializeStates deserialize_states;
|
||||||
|
|
||||||
void readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read);
|
void readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read);
|
||||||
|
|
||||||
};
|
};
|
||||||
@ -168,6 +173,12 @@ private:
|
|||||||
|
|
||||||
WriteBufferFromFile marks_stream; /// Declared below `lock` to make the file open when rwlock is captured.
|
WriteBufferFromFile marks_stream; /// Declared below `lock` to make the file open when rwlock is captured.
|
||||||
|
|
||||||
|
using SerializeState = IDataType::SerializeBinaryBulkStatePtr;
|
||||||
|
using SerializeStates = std::map<String, SerializeState>;
|
||||||
|
SerializeStates serialize_states;
|
||||||
|
|
||||||
|
IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenStreams & written_streams);
|
||||||
|
|
||||||
void writeData(const String & name, const IDataType & type, const IColumn & column,
|
void writeData(const String & name, const IDataType & type, const IColumn & column,
|
||||||
MarksForColumns & out_marks,
|
MarksForColumns & out_marks,
|
||||||
WrittenStreams & written_streams);
|
WrittenStreams & written_streams);
|
||||||
@ -226,25 +237,36 @@ Block LogBlockInputStream::readImpl()
|
|||||||
|
|
||||||
void LogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read)
|
void LogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read)
|
||||||
{
|
{
|
||||||
IDataType::InputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer *
|
IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint.
|
||||||
|
|
||||||
|
auto createStringGetter = [&](bool stream_for_prefix)
|
||||||
{
|
{
|
||||||
String stream_name = IDataType::getFileNameForStream(name, path);
|
return [&] (const IDataType::SubstreamPath & path) -> ReadBuffer *
|
||||||
|
{
|
||||||
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
||||||
|
|
||||||
const auto & file_it = storage.files.find(stream_name);
|
const auto & file_it = storage.files.find(stream_name);
|
||||||
if (storage.files.end() == file_it)
|
if (storage.files.end() == file_it)
|
||||||
throw Exception("Logical error: no information about file " + stream_name + " in StorageLog", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: no information about file " + stream_name + " in StorageLog", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
auto it = streams.try_emplace(stream_name,
|
UInt64 offset = 0;
|
||||||
file_it->second.data_file.path(),
|
if (!stream_for_prefix && mark_number)
|
||||||
mark_number
|
offset = file_it->second.marks[mark_number].offset;
|
||||||
? file_it->second.marks[mark_number].offset
|
|
||||||
: 0,
|
|
||||||
max_read_buffer_size).first;
|
|
||||||
|
|
||||||
return &it->second.compressed;
|
auto & data_file_path = file_it->second.data_file.path();
|
||||||
|
auto it = streams.try_emplace(stream_name, data_file_path, offset, max_read_buffer_size).first;
|
||||||
|
return &it->second.compressed;
|
||||||
|
};
|
||||||
};
|
};
|
||||||
|
|
||||||
type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, max_rows_to_read, 0, true, {}); /// TODO Use avg_value_size_hint.
|
if (deserialize_states.count(name) == 0)
|
||||||
|
{
|
||||||
|
settings.getter = createStringGetter(true);
|
||||||
|
type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]);
|
||||||
|
}
|
||||||
|
|
||||||
|
settings.getter = createStringGetter(false);
|
||||||
|
type.deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_states[name]);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -274,6 +296,18 @@ void LogBlockOutputStream::writeSuffix()
|
|||||||
return;
|
return;
|
||||||
done = true;
|
done = true;
|
||||||
|
|
||||||
|
WrittenStreams written_streams;
|
||||||
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
|
for (const auto & column : getHeader())
|
||||||
|
{
|
||||||
|
auto it = serialize_states.find(column.name);
|
||||||
|
if (it != serialize_states.end())
|
||||||
|
{
|
||||||
|
settings.getter = createStreamGetter(column.name, written_streams);
|
||||||
|
column.type->serializeBinaryBulkStateSuffix(settings, it->second);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// Finish write.
|
/// Finish write.
|
||||||
marks_stream.next();
|
marks_stream.next();
|
||||||
|
|
||||||
@ -291,27 +325,10 @@ void LogBlockOutputStream::writeSuffix()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void LogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column,
|
IDataType::OutputStreamGetter LogBlockOutputStream::createStreamGetter(const String & name,
|
||||||
MarksForColumns & out_marks,
|
WrittenStreams & written_streams)
|
||||||
WrittenStreams & written_streams)
|
|
||||||
{
|
{
|
||||||
type.enumerateStreams([&] (const IDataType::SubstreamPath & path)
|
return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer *
|
||||||
{
|
|
||||||
String stream_name = IDataType::getFileNameForStream(name, path);
|
|
||||||
if (written_streams.count(stream_name))
|
|
||||||
return;
|
|
||||||
|
|
||||||
const auto & file = storage.files[stream_name];
|
|
||||||
const auto stream_it = streams.try_emplace(stream_name, storage.files[stream_name].data_file.path(), storage.max_compress_block_size).first;
|
|
||||||
|
|
||||||
Mark mark;
|
|
||||||
mark.rows = (file.marks.empty() ? 0 : file.marks.back().rows) + column.size();
|
|
||||||
mark.offset = stream_it->second.plain_offset + stream_it->second.plain.count();
|
|
||||||
|
|
||||||
out_marks.emplace_back(file.column_index, mark);
|
|
||||||
}, {});
|
|
||||||
|
|
||||||
IDataType::OutputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> WriteBuffer *
|
|
||||||
{
|
{
|
||||||
String stream_name = IDataType::getFileNameForStream(name, path);
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
||||||
if (written_streams.count(stream_name))
|
if (written_streams.count(stream_name))
|
||||||
@ -319,11 +336,50 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type
|
|||||||
|
|
||||||
auto it = streams.find(stream_name);
|
auto it = streams.find(stream_name);
|
||||||
if (streams.end() == it)
|
if (streams.end() == it)
|
||||||
throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
return &it->second.compressed;
|
return &it->second.compressed;
|
||||||
};
|
};
|
||||||
|
}
|
||||||
|
|
||||||
type.serializeBinaryBulkWithMultipleStreams(column, stream_getter, 0, 0, true, {});
|
|
||||||
|
void LogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column,
|
||||||
|
MarksForColumns & out_marks,
|
||||||
|
WrittenStreams & written_streams)
|
||||||
|
{
|
||||||
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
|
|
||||||
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & path)
|
||||||
|
{
|
||||||
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
||||||
|
if (written_streams.count(stream_name))
|
||||||
|
return;
|
||||||
|
|
||||||
|
streams.try_emplace(stream_name, storage.files[stream_name].data_file.path(), storage.max_compress_block_size);
|
||||||
|
}, settings.path);
|
||||||
|
|
||||||
|
settings.getter = createStreamGetter(name, written_streams);
|
||||||
|
|
||||||
|
if (serialize_states.count(name) == 0)
|
||||||
|
type.serializeBinaryBulkStatePrefix(settings, serialize_states[name]);
|
||||||
|
|
||||||
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & path)
|
||||||
|
{
|
||||||
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
||||||
|
if (written_streams.count(stream_name))
|
||||||
|
return;
|
||||||
|
|
||||||
|
const auto & file = storage.files[stream_name];
|
||||||
|
const auto stream_it = streams.find(stream_name);
|
||||||
|
|
||||||
|
Mark mark;
|
||||||
|
mark.rows = (file.marks.empty() ? 0 : file.marks.back().rows) + column.size();
|
||||||
|
mark.offset = stream_it->second.plain_offset + stream_it->second.plain.count();
|
||||||
|
|
||||||
|
out_marks.emplace_back(file.column_index, mark);
|
||||||
|
}, settings.path);
|
||||||
|
|
||||||
|
type.serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]);
|
||||||
|
|
||||||
type.enumerateStreams([&] (const IDataType::SubstreamPath & path)
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & path)
|
||||||
{
|
{
|
||||||
@ -335,7 +391,7 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type
|
|||||||
if (streams.end() == it)
|
if (streams.end() == it)
|
||||||
throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", ErrorCodes::LOGICAL_ERROR);
|
||||||
it->second.compressed.next();
|
it->second.compressed.next();
|
||||||
}, {});
|
}, settings.path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -401,7 +457,8 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type)
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
type.enumerateStreams(stream_callback, {});
|
IDataType::SubstreamPath path;
|
||||||
|
type.enumerateStreams(stream_callback, path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -497,11 +554,12 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount() const
|
|||||||
* If this is a data type with multiple stream, get the first stream, that we assume have real row count.
|
* If this is a data type with multiple stream, get the first stream, that we assume have real row count.
|
||||||
* (Example: for Array data type, first stream is array sizes; and number of array sizes is the number of arrays).
|
* (Example: for Array data type, first stream is array sizes; and number of array sizes is the number of arrays).
|
||||||
*/
|
*/
|
||||||
|
IDataType::SubstreamPath path;
|
||||||
column_type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
column_type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
||||||
{
|
{
|
||||||
if (filename.empty())
|
if (filename.empty())
|
||||||
filename = IDataType::getFileNameForStream(column_name, substream_path);
|
filename = IDataType::getFileNameForStream(column_name, substream_path);
|
||||||
}, {});
|
}, path);
|
||||||
|
|
||||||
Files_t::const_iterator it = files.find(filename);
|
Files_t::const_iterator it = files.find(filename);
|
||||||
if (files.end() == it)
|
if (files.end() == it)
|
||||||
|
@ -95,6 +95,10 @@ private:
|
|||||||
using FileStreams = std::map<std::string, std::unique_ptr<Stream>>;
|
using FileStreams = std::map<std::string, std::unique_ptr<Stream>>;
|
||||||
FileStreams streams;
|
FileStreams streams;
|
||||||
|
|
||||||
|
using DeserializeState = IDataType::DeserializeBinaryBulkStatePtr;
|
||||||
|
using DeserializeStates = std::map<String, DeserializeState>;
|
||||||
|
DeserializeStates deserialize_states;
|
||||||
|
|
||||||
void readData(const String & name, const IDataType & type, IColumn & column, size_t limit);
|
void readData(const String & name, const IDataType & type, IColumn & column, size_t limit);
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -149,8 +153,13 @@ private:
|
|||||||
using FileStreams = std::map<std::string, std::unique_ptr<Stream>>;
|
using FileStreams = std::map<std::string, std::unique_ptr<Stream>>;
|
||||||
FileStreams streams;
|
FileStreams streams;
|
||||||
|
|
||||||
|
using SerializeState = IDataType::SerializeBinaryBulkStatePtr;
|
||||||
|
using SerializeStates = std::map<String, SerializeState>;
|
||||||
|
SerializeStates serialize_states;
|
||||||
|
|
||||||
using WrittenStreams = std::set<std::string>;
|
using WrittenStreams = std::set<std::string>;
|
||||||
|
|
||||||
|
IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenStreams & written_streams);
|
||||||
void writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams);
|
void writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams);
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -206,7 +215,8 @@ Block TinyLogBlockInputStream::readImpl()
|
|||||||
|
|
||||||
void TinyLogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t limit)
|
void TinyLogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t limit)
|
||||||
{
|
{
|
||||||
IDataType::InputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer *
|
IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint.
|
||||||
|
settings.getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer *
|
||||||
{
|
{
|
||||||
String stream_name = IDataType::getFileNameForStream(name, path);
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
||||||
|
|
||||||
@ -216,13 +226,17 @@ void TinyLogBlockInputStream::readData(const String & name, const IDataType & ty
|
|||||||
return &streams[stream_name]->compressed;
|
return &streams[stream_name]->compressed;
|
||||||
};
|
};
|
||||||
|
|
||||||
type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, limit, 0, true, {}); /// TODO Use avg_value_size_hint.
|
if (deserialize_states.count(name) == 0)
|
||||||
|
type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]);
|
||||||
|
|
||||||
|
type.deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name]);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams)
|
IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter(const String & name,
|
||||||
|
WrittenStreams & written_streams)
|
||||||
{
|
{
|
||||||
IDataType::OutputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> WriteBuffer *
|
return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer *
|
||||||
{
|
{
|
||||||
String stream_name = IDataType::getFileNameForStream(name, path);
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
||||||
|
|
||||||
@ -230,12 +244,23 @@ void TinyLogBlockOutputStream::writeData(const String & name, const IDataType &
|
|||||||
return nullptr;
|
return nullptr;
|
||||||
|
|
||||||
if (!streams.count(stream_name))
|
if (!streams.count(stream_name))
|
||||||
streams[stream_name] = std::make_unique<Stream>(storage.files[stream_name].data_file.path(), storage.max_compress_block_size);
|
streams[stream_name] = std::make_unique<Stream>(storage.files[stream_name].data_file.path(),
|
||||||
|
storage.max_compress_block_size);
|
||||||
|
|
||||||
return &streams[stream_name]->compressed;
|
return &streams[stream_name]->compressed;
|
||||||
};
|
};
|
||||||
|
}
|
||||||
|
|
||||||
type.serializeBinaryBulkWithMultipleStreams(column, stream_getter, 0, 0, true, {});
|
|
||||||
|
void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams)
|
||||||
|
{
|
||||||
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
|
settings.getter = createStreamGetter(name, written_streams);
|
||||||
|
|
||||||
|
if (serialize_states.count(name) == 0)
|
||||||
|
type.serializeBinaryBulkStatePrefix(settings, serialize_states[name]);
|
||||||
|
|
||||||
|
type.serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -249,6 +274,18 @@ void TinyLogBlockOutputStream::writeSuffix()
|
|||||||
if (streams.empty())
|
if (streams.empty())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
|
WrittenStreams written_streams;
|
||||||
|
IDataType::SerializeBinaryBulkSettings settings;
|
||||||
|
for (const auto & column : getHeader())
|
||||||
|
{
|
||||||
|
auto it = serialize_states.find(column.name);
|
||||||
|
if (it != serialize_states.end())
|
||||||
|
{
|
||||||
|
settings.getter = createStreamGetter(column.name, written_streams);
|
||||||
|
column.type->serializeBinaryBulkStateSuffix(settings, it->second);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// Finish write.
|
/// Finish write.
|
||||||
for (auto & stream : streams)
|
for (auto & stream : streams)
|
||||||
stream.second->finalize();
|
stream.second->finalize();
|
||||||
@ -324,7 +361,8 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
type.enumerateStreams(stream_callback, {});
|
IDataType::SubstreamPath path;
|
||||||
|
type.enumerateStreams(stream_callback, path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -42,7 +42,11 @@ void StorageSystemColumns::fillData(MutableColumns & res_columns, const Context
|
|||||||
/// Add `database` column.
|
/// Add `database` column.
|
||||||
MutableColumnPtr database_column_mut = ColumnString::create();
|
MutableColumnPtr database_column_mut = ColumnString::create();
|
||||||
for (const auto & database : databases)
|
for (const auto & database : databases)
|
||||||
database_column_mut->insert(database.first);
|
{
|
||||||
|
if (context.hasDatabaseAccessRights(database.first))
|
||||||
|
database_column_mut->insert(database.first);
|
||||||
|
}
|
||||||
|
|
||||||
block_to_filter.insert(ColumnWithTypeAndName(std::move(database_column_mut), std::make_shared<DataTypeString>(), "database"));
|
block_to_filter.insert(ColumnWithTypeAndName(std::move(database_column_mut), std::make_shared<DataTypeString>(), "database"));
|
||||||
|
|
||||||
/// Filter block with `database` column.
|
/// Filter block with `database` column.
|
||||||
|
@ -22,10 +22,13 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, const Contex
|
|||||||
auto databases = context.getDatabases();
|
auto databases = context.getDatabases();
|
||||||
for (const auto & database : databases)
|
for (const auto & database : databases)
|
||||||
{
|
{
|
||||||
res_columns[0]->insert(database.first);
|
if (context.hasDatabaseAccessRights(database.first))
|
||||||
res_columns[1]->insert(database.second->getEngineName());
|
{
|
||||||
res_columns[2]->insert(database.second->getDataPath());
|
res_columns[0]->insert(database.first);
|
||||||
res_columns[3]->insert(database.second->getMetadataPath());
|
res_columns[1]->insert(database.second->getEngineName());
|
||||||
|
res_columns[2]->insert(database.second->getDataPath());
|
||||||
|
res_columns[3]->insert(database.second->getMetadataPath());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -33,23 +33,26 @@ void StorageSystemMerges::fillData(MutableColumns & res_columns, const Context &
|
|||||||
{
|
{
|
||||||
for (const auto & merge : context.getMergeList().get())
|
for (const auto & merge : context.getMergeList().get())
|
||||||
{
|
{
|
||||||
size_t i = 0;
|
if (context.hasDatabaseAccessRights(merge.database))
|
||||||
res_columns[i++]->insert(merge.database);
|
{
|
||||||
res_columns[i++]->insert(merge.table);
|
size_t i = 0;
|
||||||
res_columns[i++]->insert(merge.elapsed);
|
res_columns[i++]->insert(merge.database);
|
||||||
res_columns[i++]->insert(merge.progress);
|
res_columns[i++]->insert(merge.table);
|
||||||
res_columns[i++]->insert(merge.num_parts);
|
res_columns[i++]->insert(merge.elapsed);
|
||||||
res_columns[i++]->insert(merge.source_part_names);
|
res_columns[i++]->insert(merge.progress);
|
||||||
res_columns[i++]->insert(merge.result_part_name);
|
res_columns[i++]->insert(merge.num_parts);
|
||||||
res_columns[i++]->insert(merge.total_size_bytes_compressed);
|
res_columns[i++]->insert(merge.source_part_names);
|
||||||
res_columns[i++]->insert(merge.total_size_marks);
|
res_columns[i++]->insert(merge.result_part_name);
|
||||||
res_columns[i++]->insert(merge.bytes_read_uncompressed);
|
res_columns[i++]->insert(merge.total_size_bytes_compressed);
|
||||||
res_columns[i++]->insert(merge.rows_read);
|
res_columns[i++]->insert(merge.total_size_marks);
|
||||||
res_columns[i++]->insert(merge.bytes_written_uncompressed);
|
res_columns[i++]->insert(merge.bytes_read_uncompressed);
|
||||||
res_columns[i++]->insert(merge.rows_written);
|
res_columns[i++]->insert(merge.rows_read);
|
||||||
res_columns[i++]->insert(merge.columns_written);
|
res_columns[i++]->insert(merge.bytes_written_uncompressed);
|
||||||
res_columns[i++]->insert(merge.memory_usage);
|
res_columns[i++]->insert(merge.rows_written);
|
||||||
res_columns[i++]->insert(merge.thread_number);
|
res_columns[i++]->insert(merge.columns_written);
|
||||||
|
res_columns[i++]->insert(merge.memory_usage);
|
||||||
|
res_columns[i++]->insert(merge.thread_number);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -38,12 +38,15 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex
|
|||||||
std::map<String, std::map<String, StoragePtr>> merge_tree_tables;
|
std::map<String, std::map<String, StoragePtr>> merge_tree_tables;
|
||||||
for (const auto & db : context.getDatabases())
|
for (const auto & db : context.getDatabases())
|
||||||
{
|
{
|
||||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
if (context.hasDatabaseAccessRights(db.first))
|
||||||
{
|
{
|
||||||
if (dynamic_cast<const StorageMergeTree *>(iterator->table().get())
|
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||||
|| dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
|
||||||
{
|
{
|
||||||
merge_tree_tables[db.first][iterator->name()] = iterator->table();
|
if (dynamic_cast<const StorageMergeTree *>(iterator->table().get())
|
||||||
|
|| dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||||
|
{
|
||||||
|
merge_tree_tables[db.first][iterator->name()] = iterator->table();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -59,7 +59,10 @@ public:
|
|||||||
/// Add column 'database'.
|
/// Add column 'database'.
|
||||||
MutableColumnPtr database_column_mut = ColumnString::create();
|
MutableColumnPtr database_column_mut = ColumnString::create();
|
||||||
for (const auto & database : databases)
|
for (const auto & database : databases)
|
||||||
database_column_mut->insert(database.first);
|
{
|
||||||
|
if (context.hasDatabaseAccessRights(database.first))
|
||||||
|
database_column_mut->insert(database.first);
|
||||||
|
}
|
||||||
block_to_filter.insert(ColumnWithTypeAndName(
|
block_to_filter.insert(ColumnWithTypeAndName(
|
||||||
std::move(database_column_mut), std::make_shared<DataTypeString>(), "database"));
|
std::move(database_column_mut), std::make_shared<DataTypeString>(), "database"));
|
||||||
|
|
||||||
|
@ -65,9 +65,15 @@ BlockInputStreams StorageSystemReplicas::read(
|
|||||||
/// We collect a set of replicated tables.
|
/// We collect a set of replicated tables.
|
||||||
std::map<String, std::map<String, StoragePtr>> replicated_tables;
|
std::map<String, std::map<String, StoragePtr>> replicated_tables;
|
||||||
for (const auto & db : context.getDatabases())
|
for (const auto & db : context.getDatabases())
|
||||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
{
|
||||||
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
if (context.hasDatabaseAccessRights(db.first))
|
||||||
replicated_tables[db.first][iterator->name()] = iterator->table();
|
{
|
||||||
|
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||||
|
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||||
|
replicated_tables[db.first][iterator->name()] = iterator->table();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/// Do you need columns that require a walkthrough in ZooKeeper to compute.
|
/// Do you need columns that require a walkthrough in ZooKeeper to compute.
|
||||||
bool with_zk_fields = false;
|
bool with_zk_fields = false;
|
||||||
|
@ -50,9 +50,15 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const
|
|||||||
{
|
{
|
||||||
std::map<String, std::map<String, StoragePtr>> replicated_tables;
|
std::map<String, std::map<String, StoragePtr>> replicated_tables;
|
||||||
for (const auto & db : context.getDatabases())
|
for (const auto & db : context.getDatabases())
|
||||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
{
|
||||||
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
if (context.hasDatabaseAccessRights(db.first))
|
||||||
replicated_tables[db.first][iterator->name()] = iterator->table();
|
{
|
||||||
|
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||||
|
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||||
|
replicated_tables[db.first][iterator->name()] = iterator->table();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
MutableColumnPtr col_database_mut = ColumnString::create();
|
MutableColumnPtr col_database_mut = ColumnString::create();
|
||||||
MutableColumnPtr col_table_mut = ColumnString::create();
|
MutableColumnPtr col_table_mut = ColumnString::create();
|
||||||
|
@ -95,7 +95,7 @@ BlockInputStreams StorageSystemTables::read(
|
|||||||
|
|
||||||
auto database = context.tryGetDatabase(database_name);
|
auto database = context.tryGetDatabase(database_name);
|
||||||
|
|
||||||
if (!database)
|
if (!database || !context.hasDatabaseAccessRights(database_name))
|
||||||
{
|
{
|
||||||
/// Database was deleted just now.
|
/// Database was deleted just now.
|
||||||
continue;
|
continue;
|
||||||
|
@ -11,11 +11,11 @@ class Client:
|
|||||||
self.command = [command, '--host', self.host, '--port', str(self.port), '--stacktrace']
|
self.command = [command, '--host', self.host, '--port', str(self.port), '--stacktrace']
|
||||||
|
|
||||||
|
|
||||||
def query(self, sql, stdin=None, timeout=None, settings=None):
|
def query(self, sql, stdin=None, timeout=None, settings=None, user=None):
|
||||||
return self.get_query_request(sql, stdin=stdin, timeout=timeout, settings=settings).get_answer()
|
return self.get_query_request(sql, stdin=stdin, timeout=timeout, settings=settings, user=user).get_answer()
|
||||||
|
|
||||||
|
|
||||||
def get_query_request(self, sql, stdin=None, timeout=None, settings=None):
|
def get_query_request(self, sql, stdin=None, timeout=None, settings=None, user=None):
|
||||||
command = self.command[:]
|
command = self.command[:]
|
||||||
|
|
||||||
if stdin is None:
|
if stdin is None:
|
||||||
@ -28,6 +28,9 @@ class Client:
|
|||||||
for setting, value in settings.iteritems():
|
for setting, value in settings.iteritems():
|
||||||
command += ['--' + setting, str(value)]
|
command += ['--' + setting, str(value)]
|
||||||
|
|
||||||
|
if user is not None:
|
||||||
|
command += ['--user', user]
|
||||||
|
|
||||||
return CommandRequest(command, stdin, timeout)
|
return CommandRequest(command, stdin, timeout)
|
||||||
|
|
||||||
|
|
||||||
|
@ -0,0 +1,20 @@
|
|||||||
|
<yandex>
|
||||||
|
<users>
|
||||||
|
<default>
|
||||||
|
<password></password>
|
||||||
|
<profile>default</profile>
|
||||||
|
<quota>default</quota>
|
||||||
|
</default>
|
||||||
|
<test_allow>
|
||||||
|
<password></password>
|
||||||
|
<profile>default</profile>
|
||||||
|
<quota>default</quota>
|
||||||
|
<networks>
|
||||||
|
<ip>::/0</ip>
|
||||||
|
</networks>
|
||||||
|
<allow_databases>
|
||||||
|
<database>default</database>
|
||||||
|
</allow_databases>
|
||||||
|
</test_allow>
|
||||||
|
</users>
|
||||||
|
</yandex>
|
@ -8,6 +8,7 @@ node1 = cluster.add_instance('node1', main_configs=['configs/config_no_substs.xm
|
|||||||
node2 = cluster.add_instance('node2', main_configs=['configs/config_env.xml'], env_variables={"MAX_QUERY_SIZE": "55555"})
|
node2 = cluster.add_instance('node2', main_configs=['configs/config_env.xml'], env_variables={"MAX_QUERY_SIZE": "55555"})
|
||||||
node3 = cluster.add_instance('node3', main_configs=['configs/config_zk.xml'], with_zookeeper=True)
|
node3 = cluster.add_instance('node3', main_configs=['configs/config_zk.xml'], with_zookeeper=True)
|
||||||
node4 = cluster.add_instance('node4', main_configs=['configs/config_incl.xml', 'configs/max_query_size.xml']) # include value 77777
|
node4 = cluster.add_instance('node4', main_configs=['configs/config_incl.xml', 'configs/max_query_size.xml']) # include value 77777
|
||||||
|
node5 = cluster.add_instance('node5', main_configs=['configs/config_allow_databases.xml'])
|
||||||
|
|
||||||
@pytest.fixture(scope="module")
|
@pytest.fixture(scope="module")
|
||||||
def start_cluster():
|
def start_cluster():
|
||||||
@ -26,3 +27,19 @@ def test_config(start_cluster):
|
|||||||
assert node2.query("select value from system.settings where name = 'max_query_size'") == "55555\n"
|
assert node2.query("select value from system.settings where name = 'max_query_size'") == "55555\n"
|
||||||
assert node3.query("select value from system.settings where name = 'max_query_size'") == "77777\n"
|
assert node3.query("select value from system.settings where name = 'max_query_size'") == "77777\n"
|
||||||
assert node4.query("select value from system.settings where name = 'max_query_size'") == "99999\n"
|
assert node4.query("select value from system.settings where name = 'max_query_size'") == "99999\n"
|
||||||
|
|
||||||
|
def test_allow_databases(start_cluster):
|
||||||
|
node5.query("CREATE DATABASE db1")
|
||||||
|
node5.query("CREATE TABLE db1.test_table(date Date, k1 String, v1 Int32) ENGINE = MergeTree(date, (k1, date), 8192)")
|
||||||
|
node5.query("INSERT INTO db1.test_table VALUES('2000-01-01', 'test_key', 1)")
|
||||||
|
assert node5.query("SELECT name FROM system.databases WHERE name = 'db1'") == "db1\n"
|
||||||
|
assert node5.query("SELECT name FROM system.tables WHERE database = 'db1' AND name = 'test_table' ") == "test_table\n"
|
||||||
|
assert node5.query("SELECT name FROM system.columns WHERE database = 'db1' AND table = 'test_table'") == "date\nk1\nv1\n"
|
||||||
|
assert node5.query("SELECT name FROM system.parts WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0\n"
|
||||||
|
assert node5.query("SELECT name FROM system.parts_columns WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0\n20000101_20000101_1_1_0\n20000101_20000101_1_1_0\n"
|
||||||
|
|
||||||
|
assert node5.query("SELECT name FROM system.databases WHERE name = 'db1'", user="test_allow") == "\n"
|
||||||
|
assert node5.query("SELECT name FROM system.tables WHERE database = 'db1' AND name = 'test_table'", user="test_allow") == "\n"
|
||||||
|
assert node5.query("SELECT name FROM system.columns WHERE database = 'db1' AND table = 'test_table'", user="test_allow") == "\n"
|
||||||
|
assert node5.query("SELECT name FROM system.parts WHERE database = 'db1' AND table = 'test_table'", user="test_allow") == "\n"
|
||||||
|
assert node5.query("SELECT name FROM system.parts_columns WHERE database = 'db1' AND table = 'test_table'", user="test_allow") == "\n"
|
||||||
|
@ -1 +1,16 @@
|
|||||||
SELECT DISTINCT eq FROM (WITH range(number % 10) AS arr, arrayMap(x -> x = intDiv(number, 10) ? nan : x, arr) AS arr_with_nan, arrayFilter(x -> x != intDiv(number, 10), arr) AS arr_filtered SELECT number, arrayReduce('quantileExact', arr_with_nan) AS q1, arrayReduce('quantileExact', arr_filtered) AS q2, q1 = q2 AS eq FROM numbers(100));
|
SELECT DISTINCT
|
||||||
|
eq
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
WITH
|
||||||
|
range(2 + number % 10) AS arr, -- minimum two elements, to avoid nan result --
|
||||||
|
arrayMap(x -> x = intDiv(number, 10) ? nan : x, arr) AS arr_with_nan,
|
||||||
|
arrayFilter(x -> x != intDiv(number, 10), arr) AS arr_filtered
|
||||||
|
SELECT
|
||||||
|
number,
|
||||||
|
arrayReduce('quantileExact', arr_with_nan) AS q1,
|
||||||
|
arrayReduce('quantileExact', arr_filtered) AS q2,
|
||||||
|
q1 = q2 AS eq
|
||||||
|
FROM
|
||||||
|
numbers(100)
|
||||||
|
);
|
||||||
|
@ -10,3 +10,4 @@
|
|||||||
['123']
|
['123']
|
||||||
[['1212'],['sef'],['343r4']]
|
[['1212'],['sef'],['343r4']]
|
||||||
[(1,2),(1,3),(1,5)]
|
[(1,2),(1,3),(1,5)]
|
||||||
|
0
|
||||||
|
@ -19,3 +19,4 @@ DROP TABLE arrayDistinct_test;
|
|||||||
|
|
||||||
SELECT arrayDistinct([['1212'], ['sef'], ['343r4'], ['1212']]);
|
SELECT arrayDistinct([['1212'], ['sef'], ['343r4'], ['1212']]);
|
||||||
SELECT arrayDistinct([(1, 2), (1, 3), (1, 2), (1, 2), (1, 2), (1, 5)]);
|
SELECT arrayDistinct([(1, 2), (1, 3), (1, 2), (1, 2), (1, 2), (1, 5)]);
|
||||||
|
SELECT length(arrayDistinct([NULL, NULL, NULL]));
|
||||||
|
@ -0,0 +1,6 @@
|
|||||||
|
1
|
||||||
|
2
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
21
dbms/tests/queries/0_stateless/00687_top_and_offset.sql
Normal file
21
dbms/tests/queries/0_stateless/00687_top_and_offset.sql
Normal file
@ -0,0 +1,21 @@
|
|||||||
|
DROP TABLE IF EXISTS test.test;
|
||||||
|
|
||||||
|
CREATE TABLE test.test(val Int64) engine = Memory;
|
||||||
|
|
||||||
|
INSERT INTO test.test VALUES (1);
|
||||||
|
INSERT INTO test.test VALUES (2);
|
||||||
|
INSERT INTO test.test VALUES (3);
|
||||||
|
INSERT INTO test.test VALUES (4);
|
||||||
|
INSERT INTO test.test VALUES (5);
|
||||||
|
INSERT INTO test.test VALUES (6);
|
||||||
|
INSERT INTO test.test VALUES (7);
|
||||||
|
INSERT INTO test.test VALUES (8);
|
||||||
|
INSERT INTO test.test VALUES (9);
|
||||||
|
|
||||||
|
SELECT TOP 2 * FROM test.test ORDER BY val;
|
||||||
|
SELECT TOP (2) * FROM test.test ORDER BY val;
|
||||||
|
SELECT * FROM test.test ORDER BY val LIMIT 2 OFFSET 2;
|
||||||
|
SELECT TOP 2 * FROM test.test ORDER BY val LIMIT 2; -- { clientError 406 }
|
||||||
|
SELECT * FROM test.test ORDER BY val LIMIT 2,3 OFFSET 2; -- { clientError 62 }
|
||||||
|
|
||||||
|
DROP TABLE test.test;
|
@ -0,0 +1,4 @@
|
|||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
@ -0,0 +1,4 @@
|
|||||||
|
select CAST(toLowCardinality(val) as UInt64) from (select arrayJoin(['1']) as val);
|
||||||
|
select toUInt64(toLowCardinality(val)) from (select arrayJoin(['1']) as val);
|
||||||
|
select 1 % toLowCardinality(val) from (select arrayJoin([1]) as val);
|
||||||
|
select gcd(1, toLowCardinality(val)) from (select arrayJoin([1]) as val);
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user