2018-09-27 15:55:22 +00:00
|
|
|
#include <Columns/ColumnLowCardinality.h>
|
2018-07-09 18:19:03 +00:00
|
|
|
#include <Columns/ColumnsNumber.h>
|
2018-03-19 11:45:17 +00:00
|
|
|
#include <DataStreams/ColumnGathererStream.h>
|
2018-07-09 18:19:03 +00:00
|
|
|
#include <DataTypes/NumberTraits.h>
|
|
|
|
#include <Common/HashTable/HashMap.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
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>
|
2018-08-06 13:12:45 +00:00
|
|
|
MutableColumnPtr mapUniqueIndexImplRef(PaddedPODArray<T> & index)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
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)
|
2019-02-28 09:35:38 +00:00
|
|
|
data[val.getSecond()] = val.getFirst();
|
2018-07-09 18:19:03 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
|
Get rid of useless std::move to get NRVO
http://eel.is/c++draft/class.copy.elision#:constructor,copy,elision
Some quote:
> Speaking of RVO, return std::move(w); prohibits it. It means "use move constructor or fail to compile", whereas return w; means "use RVO, and if you can't, use move constructor, and if you can't, use copy constructor, and if you can't, fail to compile."
There is one exception to this rule:
```cpp
Block FilterBlockInputStream::removeFilterIfNeed(Block && block)
{
if (block && remove_filter)
block.erase(static_cast<size_t>(filter_column));
return std::move(block);
}
```
because references are not eligible for NRVO, which is another rule "always move rvalue references and forward universal references" that takes precedence.
2018-08-27 14:04:22 +00:00
|
|
|
return res_col;
|
2018-07-09 18:19:03 +00:00
|
|
|
}
|
|
|
|
|
2018-08-06 13:12:45 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
Get rid of useless std::move to get NRVO
http://eel.is/c++draft/class.copy.elision#:constructor,copy,elision
Some quote:
> Speaking of RVO, return std::move(w); prohibits it. It means "use move constructor or fail to compile", whereas return w; means "use RVO, and if you can't, use move constructor, and if you can't, use copy constructor, and if you can't, fail to compile."
There is one exception to this rule:
```cpp
Block FilterBlockInputStream::removeFilterIfNeed(Block && block)
{
if (block && remove_filter)
block.erase(static_cast<size_t>(filter_column));
return std::move(block);
}
```
because references are not eligible for NRVO, which is another rule "always move rvalue references and forward universal references" that takes precedence.
2018-08-27 14:04:22 +00:00
|
|
|
return res_col;
|
2018-08-06 13:12:45 +00:00
|
|
|
}
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
/// 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);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
ColumnLowCardinality::ColumnLowCardinality(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_, bool is_shared)
|
2018-09-21 14:15:21 +00:00
|
|
|
: dictionary(std::move(column_unique_), is_shared), idx(std::move(indexes_))
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
idx.check(getDictionary().size());
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::insert(const Field & x)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
compactIfSharedDictionary();
|
|
|
|
idx.insertPosition(dictionary.getColumnUnique().uniqueInsert(x));
|
|
|
|
idx.check(getDictionary().size());
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::insertDefault()
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
idx.insertPosition(getDictionary().getDefaultValueIndex());
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::insertFrom(const IColumn & src, size_t n)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
2018-09-27 15:55:22 +00:00
|
|
|
auto * low_cardinality_src = typeid_cast<const ColumnLowCardinality *>(&src);
|
2018-07-09 18:19:03 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
if (!low_cardinality_src)
|
|
|
|
throw Exception("Expected ColumnLowCardinality, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
2018-07-09 18:19:03 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
size_t position = low_cardinality_src->getIndexes().getUInt(n);
|
2018-07-09 18:19:03 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
if (&low_cardinality_src->getDictionary() == &getDictionary())
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
/// Dictionary is shared with src column. Insert only index.
|
|
|
|
idx.insertPosition(position);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
compactIfSharedDictionary();
|
2018-09-27 15:55:22 +00:00
|
|
|
const auto & nested = *low_cardinality_src->getDictionary().getNestedColumn();
|
2018-07-09 18:19:03 +00:00
|
|
|
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(nested, position));
|
|
|
|
}
|
|
|
|
|
|
|
|
idx.check(getDictionary().size());
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::insertFromFullColumn(const IColumn & src, size_t n)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
compactIfSharedDictionary();
|
|
|
|
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(src, n));
|
|
|
|
idx.check(getDictionary().size());
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::insertRangeFrom(const IColumn & src, size_t start, size_t length)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
2018-09-27 15:55:22 +00:00
|
|
|
auto * low_cardinality_src = typeid_cast<const ColumnLowCardinality *>(&src);
|
2018-07-09 18:19:03 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
if (!low_cardinality_src)
|
2018-12-06 11:24:07 +00:00
|
|
|
throw Exception("Expected ColumnLowCardinality, got " + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
2018-07-09 18:19:03 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
if (&low_cardinality_src->getDictionary() == &getDictionary())
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
/// Dictionary is shared with src column. Insert only indexes.
|
2018-09-27 15:55:22 +00:00
|
|
|
idx.insertPositionsRange(low_cardinality_src->getIndexes(), start, length);
|
2018-07-09 18:19:03 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
compactIfSharedDictionary();
|
|
|
|
|
|
|
|
/// TODO: Support native insertion from other unique column. It will help to avoid null map creation.
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
auto sub_idx = (*low_cardinality_src->getIndexes().cut(start, length)).mutate();
|
2018-07-09 18:19:03 +00:00
|
|
|
auto idx_map = mapUniqueIndex(*sub_idx);
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
auto src_nested = low_cardinality_src->getDictionary().getNestedColumn();
|
2018-07-09 18:19:03 +00:00
|
|
|
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());
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
compactIfSharedDictionary();
|
|
|
|
auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(src, start, length);
|
|
|
|
idx.insertPositionsRange(*inserted_indexes, 0, length);
|
|
|
|
idx.check(getDictionary().size());
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
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());
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::insertData(const char * pos, size_t length)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
compactIfSharedDictionary();
|
|
|
|
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertData(pos, length));
|
|
|
|
idx.check(getDictionary().size());
|
|
|
|
}
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
StringRef ColumnLowCardinality::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin);
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
const char * ColumnLowCardinality::deserializeAndInsertFromArena(const char * pos)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
compactIfSharedDictionary();
|
|
|
|
|
|
|
|
const char * new_pos;
|
|
|
|
idx.insertPosition(dictionary.getColumnUnique().uniqueDeserializeAndInsertFromArena(pos, new_pos));
|
|
|
|
|
|
|
|
idx.check(getDictionary().size());
|
|
|
|
return new_pos;
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::gather(ColumnGathererStream & gatherer)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
|
|
|
gatherer.gather(*this);
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
auto unique_ptr = dictionary.getColumnUniquePtr();
|
2018-12-28 15:22:38 +00:00
|
|
|
if (size == 0)
|
|
|
|
unique_ptr = unique_ptr->cloneEmpty();
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
return ColumnLowCardinality::create((*std::move(unique_ptr)).mutate(), getIndexes().cloneResized(size));
|
2018-07-09 18:19:03 +00:00
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
const auto & low_cardinality_column = assert_cast<const ColumnLowCardinality &>(rhs);
|
2018-07-09 18:19:03 +00:00
|
|
|
size_t n_index = getIndexes().getUInt(n);
|
2018-09-27 15:55:22 +00:00
|
|
|
size_t m_index = low_cardinality_column.getIndexes().getUInt(m);
|
|
|
|
return getDictionary().compareAt(n_index, m_index, low_cardinality_column.getDictionary(), nan_direction_hint);
|
2018-07-09 18:19:03 +00:00
|
|
|
}
|
|
|
|
|
2019-02-18 19:44:26 +00:00
|
|
|
void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
if (limit == 0)
|
|
|
|
limit = size();
|
|
|
|
|
2018-11-27 18:08:54 +00:00
|
|
|
size_t unique_limit = getDictionary().size();
|
2018-07-09 18:19:03 +00:00
|
|
|
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;
|
2018-09-17 08:18:46 +00:00
|
|
|
for (size_t row = 0; row < unique_perm.size() && perm_index < perm_size; ++row)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
std::vector<MutableColumnPtr> ColumnLowCardinality::scatter(ColumnIndex num_columns, const Selector & selector) const
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
auto columns = getIndexes().scatter(num_columns, selector);
|
|
|
|
for (auto & column : columns)
|
|
|
|
{
|
|
|
|
auto unique_ptr = dictionary.getColumnUniquePtr();
|
2018-09-27 15:55:22 +00:00
|
|
|
column = ColumnLowCardinality::create((*std::move(unique_ptr)).mutate(), std::move(column));
|
2018-07-09 18:19:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return columns;
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::setSharedDictionary(const ColumnPtr & column_unique)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
if (!empty())
|
2018-09-27 15:55:22 +00:00
|
|
|
throw Exception("Can't set ColumnUnique for ColumnLowCardinality because is't not empty.",
|
2018-07-09 18:19:03 +00:00
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
dictionary.setShared(column_unique);
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
ColumnLowCardinality::MutablePtr ColumnLowCardinality::cutAndCompact(size_t start, size_t length) const
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
auto sub_positions = (*idx.getPositions()->cut(start, length)).mutate();
|
|
|
|
/// Create column with new indexes and old dictionary.
|
2019-03-29 15:07:53 +00:00
|
|
|
/// Dictionary is shared, but will be recreated after compactInplace call.
|
2018-09-27 15:55:22 +00:00
|
|
|
auto column = ColumnLowCardinality::create(getDictionary().assumeMutable(), std::move(sub_positions));
|
2018-07-09 18:19:03 +00:00
|
|
|
/// Will create new dictionary.
|
|
|
|
column->compactInplace();
|
|
|
|
|
|
|
|
return column;
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::compactInplace()
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
auto positions = idx.detachPositions();
|
|
|
|
dictionary.compact(positions);
|
|
|
|
idx.attachPositions(std::move(positions));
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::compactIfSharedDictionary()
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
if (dictionary.isShared())
|
|
|
|
compactInplace();
|
2018-06-28 18:12:29 +00:00
|
|
|
}
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
ColumnLowCardinality::DictionaryEncodedColumn
|
2019-02-10 16:22:38 +00:00
|
|
|
ColumnLowCardinality::getMinimalDictionaryEncodedColumn(UInt64 offset, UInt64 limit) const
|
2018-07-19 19:12:48 +00:00
|
|
|
{
|
|
|
|
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)};
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
ColumnPtr ColumnLowCardinality::countKeys() const
|
2018-08-13 16:23:40 +00:00
|
|
|
{
|
|
|
|
const auto & nested_column = getDictionary().getNestedColumn();
|
|
|
|
size_t dict_size = nested_column->size();
|
|
|
|
|
|
|
|
auto counter = ColumnUInt64::create(dict_size, 0);
|
|
|
|
idx.countKeys(counter->getData());
|
2019-05-31 08:55:01 +00:00
|
|
|
return counter;
|
2018-08-13 16:23:40 +00:00
|
|
|
}
|
|
|
|
|
2019-07-22 15:41:52 +00:00
|
|
|
bool ColumnLowCardinality::containsNull() const
|
|
|
|
{
|
|
|
|
return getDictionary().nestedColumnIsNullable() && idx.containsDefault();
|
|
|
|
}
|
|
|
|
|
2018-08-13 16:23:40 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
ColumnLowCardinality::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {}
|
2018-07-09 18:19:03 +00:00
|
|
|
|
2019-08-03 11:02:40 +00:00
|
|
|
ColumnLowCardinality::Index::Index(MutableColumnPtr && positions_) : positions(std::move(positions_))
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
updateSizeOfType();
|
|
|
|
}
|
|
|
|
|
2019-08-03 11:02:40 +00:00
|
|
|
ColumnLowCardinality::Index::Index(ColumnPtr positions_) : positions(std::move(positions_))
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
updateSizeOfType();
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Callback>
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Index::callForType(Callback && callback, size_t size_of_type)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
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: {
|
2018-09-27 15:55:22 +00:00
|
|
|
throw Exception("Unexpected size of index type for ColumnLowCardinality: " + toString(size_of_type),
|
2018-07-09 18:19:03 +00:00
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
size_t ColumnLowCardinality::Index::getSizeOfIndexType(const IColumn & column, size_t hint)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
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);
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
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;
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
throw Exception("Unexpected indexes type for ColumnLowCardinality. Expected UInt, got " + column.getName(),
|
2018-06-28 18:12:29 +00:00
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Index::attachPositions(ColumnPtr positions_)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
positions = std::move(positions_);
|
|
|
|
updateSizeOfType();
|
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename IndexType>
|
2018-09-27 15:55:22 +00:00
|
|
|
typename ColumnVector<IndexType>::Container & ColumnLowCardinality::Index::getPositionsData()
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
auto * positions_ptr = typeid_cast<ColumnVector<IndexType> *>(positions->assumeMutable().get());
|
|
|
|
if (!positions_ptr)
|
2018-09-27 15:55:22 +00:00
|
|
|
throw Exception("Invalid indexes type for ColumnLowCardinality."
|
2018-07-09 18:19:03 +00:00
|
|
|
" Expected UInt" + toString(8 * sizeof(IndexType)) + ", got " + positions->getName(),
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
return positions_ptr->getData();
|
|
|
|
}
|
|
|
|
|
2018-08-13 16:23:40 +00:00
|
|
|
template <typename IndexType>
|
2018-09-27 15:55:22 +00:00
|
|
|
const typename ColumnVector<IndexType>::Container & ColumnLowCardinality::Index::getPositionsData() const
|
2018-08-13 16:23:40 +00:00
|
|
|
{
|
|
|
|
const auto * positions_ptr = typeid_cast<const ColumnVector<IndexType> *>(positions.get());
|
|
|
|
if (!positions_ptr)
|
2018-09-27 15:55:22 +00:00
|
|
|
throw Exception("Invalid indexes type for ColumnLowCardinality."
|
2018-08-13 16:23:40 +00:00
|
|
|
" Expected UInt" + toString(8 * sizeof(IndexType)) + ", got " + positions->getName(),
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
return positions_ptr->getData();
|
|
|
|
}
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
template <typename IndexType>
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Index::convertPositions()
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
|
|
|
auto convert = [&](auto x)
|
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
using CurIndexType = decltype(x);
|
|
|
|
auto & data = getPositionsData<CurIndexType>();
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
if (sizeof(CurIndexType) > sizeof(IndexType))
|
|
|
|
throw Exception("Converting indexes to smaller type: from " + toString(sizeof(CurIndexType)) +
|
|
|
|
" to " + toString(sizeof(IndexType)), ErrorCodes::LOGICAL_ERROR);
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
if (sizeof(CurIndexType) != sizeof(IndexType))
|
|
|
|
{
|
|
|
|
size_t size = data.size();
|
|
|
|
auto new_positions = ColumnVector<IndexType>::create(size);
|
|
|
|
auto & new_data = new_positions->getData();
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
/// 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);
|
2018-06-28 18:12:29 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
callForType(std::move(convert), size_of_type);
|
|
|
|
|
|
|
|
checkSizeOfType();
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Index::expandType()
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
auto expand = [&](auto type)
|
|
|
|
{
|
|
|
|
using CurIndexType = decltype(type);
|
|
|
|
constexpr auto next_size = NumberTraits::nextSize(sizeof(CurIndexType));
|
|
|
|
if (next_size == sizeof(CurIndexType))
|
2018-09-27 15:55:22 +00:00
|
|
|
throw Exception("Can't expand indexes type for ColumnLowCardinality from type: "
|
2018-07-09 18:19:03 +00:00
|
|
|
+ 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);
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
UInt64 ColumnLowCardinality::Index::getMaxPositionForCurrentType() const
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
UInt64 value = 0;
|
|
|
|
callForType([&](auto type) { value = std::numeric_limits<decltype(type)>::max(); }, size_of_type);
|
|
|
|
return value;
|
2018-06-28 18:12:29 +00:00
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
size_t ColumnLowCardinality::Index::getPositionAt(size_t row) const
|
2018-08-23 13:22:03 +00:00
|
|
|
{
|
|
|
|
size_t pos;
|
|
|
|
auto getPosition = [&](auto type)
|
|
|
|
{
|
|
|
|
using CurIndexType = decltype(type);
|
|
|
|
pos = getPositionsData<CurIndexType>()[row];
|
|
|
|
};
|
|
|
|
|
|
|
|
callForType(std::move(getPosition), size_of_type);
|
|
|
|
return pos;
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Index::insertPosition(UInt64 position)
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
while (position > getMaxPositionForCurrentType())
|
|
|
|
expandType();
|
|
|
|
|
2019-03-25 01:43:54 +00:00
|
|
|
positions->insert(position);
|
2018-07-09 18:19:03 +00:00
|
|
|
checkSizeOfType();
|
|
|
|
}
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2019-02-10 16:22:38 +00:00
|
|
|
void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, UInt64 offset, UInt64 limit)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
auto insertForType = [&](auto type)
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
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))
|
2019-03-25 01:43:54 +00:00
|
|
|
positions->insertRangeFrom(column, offset, limit);
|
2018-07-09 18:19:03 +00:00
|
|
|
else
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
auto copy = [&](auto cur_type)
|
|
|
|
{
|
|
|
|
using CurIndexType = decltype(cur_type);
|
|
|
|
auto & positions_data = getPositionsData<CurIndexType>();
|
|
|
|
const auto & column_data = column_ptr->getData();
|
|
|
|
|
2019-02-10 16:22:38 +00:00
|
|
|
UInt64 size = positions_data.size();
|
2018-07-09 18:19:03 +00:00
|
|
|
positions_data.resize(size + limit);
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2019-02-10 16:22:38 +00:00
|
|
|
for (UInt64 i = 0; i < limit; ++i)
|
2018-07-09 18:19:03 +00:00
|
|
|
positions_data[size + i] = column_data[offset + i];
|
|
|
|
};
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
callForType(std::move(copy), size_of_type);
|
2018-06-28 18:12:29 +00:00
|
|
|
}
|
2018-07-09 18:19:03 +00:00
|
|
|
|
|
|
|
return true;
|
2018-06-28 18:12:29 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
if (!insertForType(UInt8()) &&
|
|
|
|
!insertForType(UInt16()) &&
|
|
|
|
!insertForType(UInt32()) &&
|
|
|
|
!insertForType(UInt64()))
|
2018-09-27 15:55:22 +00:00
|
|
|
throw Exception("Invalid column for ColumnLowCardinality index. Expected UInt, got " + column.getName(),
|
2018-07-09 18:19:03 +00:00
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
|
|
|
checkSizeOfType();
|
2018-06-28 18:12:29 +00:00
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Index::check(size_t /*max_dictionary_size*/)
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
2018-07-20 16:51:44 +00:00
|
|
|
/// TODO: remove
|
|
|
|
/*
|
2018-07-09 18:19:03 +00:00
|
|
|
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);
|
2018-07-20 16:51:44 +00:00
|
|
|
*/
|
2018-07-09 18:19:03 +00:00
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Index::checkSizeOfType()
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
if (size_of_type != getSizeOfIndexType(*positions, size_of_type))
|
2018-11-26 00:56:50 +00:00
|
|
|
throw Exception("Invalid size of type. Expected " + toString(8 * size_of_type) +
|
2018-07-09 18:19:03 +00:00
|
|
|
", but positions are " + positions->getName(), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Index::countKeys(ColumnUInt64::Container & counts) const
|
2018-08-21 14:53:51 +00:00
|
|
|
{
|
|
|
|
auto counter = [&](auto x)
|
|
|
|
{
|
|
|
|
using CurIndexType = decltype(x);
|
|
|
|
auto & data = getPositionsData<CurIndexType>();
|
|
|
|
for (auto pos : data)
|
|
|
|
++counts[pos];
|
|
|
|
};
|
|
|
|
callForType(std::move(counter), size_of_type);
|
|
|
|
}
|
|
|
|
|
2019-07-22 15:41:52 +00:00
|
|
|
bool ColumnLowCardinality::Index::containsDefault() const
|
|
|
|
{
|
|
|
|
bool contains = false;
|
|
|
|
|
2019-07-22 16:46:42 +00:00
|
|
|
auto check_contains_default = [&](auto x)
|
2019-07-22 15:41:52 +00:00
|
|
|
{
|
|
|
|
using CurIndexType = decltype(x);
|
|
|
|
auto & data = getPositionsData<CurIndexType>();
|
|
|
|
for (auto pos : data)
|
|
|
|
{
|
|
|
|
if (pos == 0)
|
|
|
|
{
|
|
|
|
contains = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
callForType(std::move(check_contains_default), size_of_type);
|
|
|
|
return contains;
|
|
|
|
}
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
ColumnLowCardinality::Dictionary::Dictionary(MutableColumnPtr && column_unique_, bool is_shared)
|
2018-09-21 14:15:21 +00:00
|
|
|
: column_unique(std::move(column_unique_)), shared(is_shared)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
checkColumn(*column_unique);
|
|
|
|
}
|
2018-09-27 15:55:22 +00:00
|
|
|
ColumnLowCardinality::Dictionary::Dictionary(ColumnPtr column_unique_, bool is_shared)
|
2018-09-21 14:15:21 +00:00
|
|
|
: column_unique(std::move(column_unique_)), shared(is_shared)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
checkColumn(*column_unique);
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Dictionary::checkColumn(const IColumn & column)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
|
|
|
|
if (!dynamic_cast<const IColumnUnique *>(&column))
|
2018-09-27 15:55:22 +00:00
|
|
|
throw Exception("ColumnUnique expected as an argument of ColumnLowCardinality.", ErrorCodes::ILLEGAL_COLUMN);
|
2018-07-09 18:19:03 +00:00
|
|
|
}
|
|
|
|
|
2019-01-04 12:10:00 +00:00
|
|
|
void ColumnLowCardinality::Dictionary::setShared(const ColumnPtr & column_unique_)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
2019-01-04 12:10:00 +00:00
|
|
|
checkColumn(*column_unique_);
|
2018-07-09 18:19:03 +00:00
|
|
|
|
2019-01-04 12:10:00 +00:00
|
|
|
column_unique = column_unique_;
|
2018-07-09 18:19:03 +00:00
|
|
|
shared = true;
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
void ColumnLowCardinality::Dictionary::compact(ColumnPtr & positions)
|
2018-07-09 18:19:03 +00:00
|
|
|
{
|
|
|
|
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);
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
shared = false;
|
2018-06-28 18:12:29 +00:00
|
|
|
}
|
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|